From bbce8b417a0b9827c7ee947ba8025d8ebc6b8f0a Mon Sep 17 00:00:00 2001 From: Yuanjian Li Date: Thu, 29 Aug 2019 18:02:38 +0800 Subject: [PATCH 01/15] Use the taskAttemptId in ShuffleId --- .../spark/network/protocol/Encoders.java | 23 +++ .../network/shuffle/ExternalBlockHandler.java | 4 +- .../shuffle/ExternalShuffleBlockResolver.java | 4 +- .../shuffle/OneForOneBlockFetcher.java | 27 +-- .../shuffle/protocol/FetchShuffleBlocks.java | 10 +- .../shuffle/BlockTransferMessagesSuite.java | 2 +- .../shuffle/ExternalBlockHandlerSuite.java | 2 +- .../shuffle/OneForOneBlockFetcherSuite.java | 4 +- .../api/ShuffleExecutorComponents.java | 5 - .../sort/BypassMergeSortShuffleWriter.java | 11 +- .../shuffle/sort/UnsafeShuffleWriter.java | 12 +- .../LocalDiskShuffleExecutorComponents.java | 6 +- .../io/LocalDiskShuffleMapOutputWriter.java | 11 +- .../LocalDiskSingleSpillMapOutputWriter.java | 4 +- .../scala/org/apache/spark/Dependency.scala | 2 +- .../org/apache/spark/MapOutputTracker.scala | 4 +- .../spark/internal/config/package.scala | 8 + .../apache/spark/scheduler/DAGScheduler.scala | 30 ++- .../apache/spark/scheduler/MapStatus.scala | 48 +++-- .../spark/scheduler/ShuffleMapTask.scala | 2 +- .../org/apache/spark/scheduler/Stage.scala | 6 +- .../spark/shuffle/BaseShuffleHandle.scala | 1 - .../shuffle/IndexShuffleBlockResolver.scala | 8 +- .../apache/spark/shuffle/ShuffleManager.scala | 2 - .../spark/shuffle/ShuffleWriteProcessor.scala | 2 - .../shuffle/sort/SortShuffleManager.scala | 34 ++-- .../shuffle/sort/SortShuffleWriter.scala | 8 +- .../org/apache/spark/storage/BlockId.scala | 12 +- .../util/collection/ExternalSorter.scala | 2 +- .../sort/UnsafeShuffleWriterSuite.java | 12 +- .../apache/spark/MapOutputTrackerSuite.scala | 38 ++-- .../scala/org/apache/spark/ShuffleSuite.scala | 6 +- .../spark/memory/MemoryTestingUtils.scala | 4 +- .../spark/scheduler/DAGSchedulerSuite.scala | 184 ++++++++++++++---- .../spark/scheduler/MapStatusSuite.scala | 12 +- .../serializer/KryoSerializerSuite.scala | 5 +- .../BlockStoreShuffleReaderSuite.scala | 2 +- .../BypassMergeSortShuffleWriterSuite.scala | 9 +- .../shuffle/sort/SortShuffleWriterSuite.scala | 8 +- ...LocalDiskShuffleMapOutputWriterSuite.scala | 6 +- project/MimaExcludes.scala | 20 +- 41 files changed, 388 insertions(+), 212 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 736059fdd1f57..490915f6de4b3 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 @@ -112,4 +112,27 @@ public static int[] decode(ByteBuf buf) { return ints; } } + + /** Long integer arrays are encoded with their length followed by long integers. */ + public static class LongArrays { + public static int encodedLength(long[] longs) { + return 4 + 8 * longs.length; + } + + public static void encode(ByteBuf buf, long[] longs) { + buf.writeInt(longs.length); + for (long i : longs) { + buf.writeLong(i); + } + } + + public static long[] decode(ByteBuf buf) { + int numLongs = buf.readInt(); + long[] longs = new long[numLongs]; + for (int i = 0; i < longs.length; i ++) { + longs[i] = buf.readLong(); + } + return longs; + } + } } 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 037e5cf7e5222..9ba78808415e8 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 @@ -300,7 +300,7 @@ private int[] shuffleMapIdAndReduceIds(String[] blockIds, int shuffleId) { } ManagedBufferIterator(FetchShuffleBlocks msg, int numBlockIds) { - final int[] mapIdAndReduceIds = new int[2 * numBlockIds]; + final long[] mapIdAndReduceIds = new long[2 * numBlockIds]; int idx = 0; for (int i = 0; i < msg.mapIds.length; i++) { for (int reduceId : msg.reduceIds[i]) { @@ -311,7 +311,7 @@ private int[] shuffleMapIdAndReduceIds(String[] blockIds, int shuffleId) { assert(idx == 2 * numBlockIds); size = mapIdAndReduceIds.length; blockDataForIndexFn = index -> blockManager.getBlockData(msg.appId, msg.execId, - msg.shuffleId, mapIdAndReduceIds[index], mapIdAndReduceIds[index + 1]); + msg.shuffleId, mapIdAndReduceIds[index], (int) mapIdAndReduceIds[index + 1]); } @Override 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 50f16fc700f12..8b0d1e145a813 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 @@ -172,7 +172,7 @@ public ManagedBuffer getBlockData( String appId, String execId, int shuffleId, - int mapId, + long mapId, int reduceId) { ExecutorShuffleInfo executor = executors.get(new AppExecId(appId, execId)); if (executor == null) { @@ -296,7 +296,7 @@ private void deleteNonShuffleServiceServedFiles(String[] dirs) { * and the block id format is from ShuffleDataBlockId and ShuffleIndexBlockId. */ private ManagedBuffer getSortBasedShuffleBlockData( - ExecutorShuffleInfo executor, int shuffleId, int mapId, int reduceId) { + ExecutorShuffleInfo executor, int shuffleId, long mapId, int reduceId) { File indexFile = ExecutorDiskUtils.getFile(executor.localDirs, executor.subDirsPerLocalDir, "shuffle_" + shuffleId + "_" + mapId + "_0.index"); diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java index cc11e92067375..52854c86be3e6 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java @@ -24,6 +24,8 @@ import java.util.HashMap; import com.google.common.primitives.Ints; +import com.google.common.primitives.Longs; +import org.apache.commons.lang3.tuple.ImmutableTriple; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -111,21 +113,21 @@ private boolean isShuffleBlocks(String[] blockIds) { */ private FetchShuffleBlocks createFetchShuffleBlocksMsg( String appId, String execId, String[] blockIds) { - int shuffleId = splitBlockId(blockIds[0])[0]; - HashMap> mapIdToReduceIds = new HashMap<>(); + int shuffleId = splitBlockId(blockIds[0]).left; + HashMap> mapIdToReduceIds = new HashMap<>(); for (String blockId : blockIds) { - int[] blockIdParts = splitBlockId(blockId); - if (blockIdParts[0] != shuffleId) { + ImmutableTriple blockIdParts = splitBlockId(blockId); + if (blockIdParts.left != shuffleId) { throw new IllegalArgumentException("Expected shuffleId=" + shuffleId + ", got:" + blockId); } - int mapId = blockIdParts[1]; + long mapId = blockIdParts.middle; if (!mapIdToReduceIds.containsKey(mapId)) { mapIdToReduceIds.put(mapId, new ArrayList<>()); } - mapIdToReduceIds.get(mapId).add(blockIdParts[2]); + mapIdToReduceIds.get(mapId).add(blockIdParts.right); } - int[] mapIds = Ints.toArray(mapIdToReduceIds.keySet()); + long[] mapIds = Longs.toArray(mapIdToReduceIds.keySet()); int[][] reduceIdArr = new int[mapIds.length][]; for (int i = 0; i < mapIds.length; i++) { reduceIdArr[i] = Ints.toArray(mapIdToReduceIds.get(mapIds[i])); @@ -134,17 +136,16 @@ private FetchShuffleBlocks createFetchShuffleBlocksMsg( } /** Split the shuffleBlockId and return shuffleId, mapId and reduceId. */ - private int[] splitBlockId(String blockId) { + private ImmutableTriple splitBlockId(String blockId) { String[] blockIdParts = blockId.split("_"); if (blockIdParts.length != 4 || !blockIdParts[0].equals("shuffle")) { throw new IllegalArgumentException( "Unexpected shuffle block id format: " + blockId); } - return new int[] { - Integer.parseInt(blockIdParts[1]), - Integer.parseInt(blockIdParts[2]), - Integer.parseInt(blockIdParts[3]) - }; + return new ImmutableTriple<>( + Integer.parseInt(blockIdParts[1]), + Long.parseLong(blockIdParts[2]), + Integer.parseInt(blockIdParts[3])); } /** Callback invoked on receipt of each chunk. We equate a single chunk to a single block. */ diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/FetchShuffleBlocks.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/FetchShuffleBlocks.java index 466eeb3e048a8..faa960d414bcc 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/FetchShuffleBlocks.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/FetchShuffleBlocks.java @@ -34,14 +34,14 @@ public class FetchShuffleBlocks extends BlockTransferMessage { public final int shuffleId; // The length of mapIds must equal to reduceIds.size(), for the i-th mapId in mapIds, // it corresponds to the i-th int[] in reduceIds, which contains all reduce id for this map id. - public final int[] mapIds; + public final long[] mapIds; public final int[][] reduceIds; public FetchShuffleBlocks( String appId, String execId, int shuffleId, - int[] mapIds, + long[] mapIds, int[][] reduceIds) { this.appId = appId; this.execId = execId; @@ -98,7 +98,7 @@ public int encodedLength() { return Encoders.Strings.encodedLength(appId) + Encoders.Strings.encodedLength(execId) + 4 /* encoded length of shuffleId */ - + Encoders.IntArrays.encodedLength(mapIds) + + Encoders.LongArrays.encodedLength(mapIds) + 4 /* encoded length of reduceIds.size() */ + encodedLengthOfReduceIds; } @@ -108,7 +108,7 @@ public void encode(ByteBuf buf) { Encoders.Strings.encode(buf, appId); Encoders.Strings.encode(buf, execId); buf.writeInt(shuffleId); - Encoders.IntArrays.encode(buf, mapIds); + Encoders.LongArrays.encode(buf, mapIds); buf.writeInt(reduceIds.length); for (int[] ids: reduceIds) { Encoders.IntArrays.encode(buf, ids); @@ -119,7 +119,7 @@ public static FetchShuffleBlocks decode(ByteBuf buf) { String appId = Encoders.Strings.decode(buf); String execId = Encoders.Strings.decode(buf); int shuffleId = buf.readInt(); - int[] mapIds = Encoders.IntArrays.decode(buf); + long[] mapIds = Encoders.LongArrays.decode(buf); int reduceIdsSize = buf.readInt(); int[][] reduceIds = new int[reduceIdsSize][]; for (int i = 0; i < reduceIdsSize; i++) { diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/BlockTransferMessagesSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/BlockTransferMessagesSuite.java index 649c471dc1679..ba40f4a45ac8f 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/BlockTransferMessagesSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/BlockTransferMessagesSuite.java @@ -29,7 +29,7 @@ public class BlockTransferMessagesSuite { public void serializeOpenShuffleBlocks() { checkSerializeDeserialize(new OpenBlocks("app-1", "exec-2", new String[] { "b1", "b2" })); checkSerializeDeserialize(new FetchShuffleBlocks( - "app-1", "exec-2", 0, new int[] {0, 1}, + "app-1", "exec-2", 0, new long[] {0, 1}, new int[][] {{ 0, 1 }, { 0, 1, 2 }})); checkSerializeDeserialize(new RegisterExecutor("app-1", "exec-2", new ExecutorShuffleInfo( new String[] { "/local1", "/local2" }, 32, "MyShuffleManager"))); 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 9c623a70424b6..6a5d04b6f417b 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 @@ -101,7 +101,7 @@ public void testFetchShuffleBlocks() { when(blockResolver.getBlockData("app0", "exec1", 0, 0, 1)).thenReturn(blockMarkers[1]); FetchShuffleBlocks fetchShuffleBlocks = new FetchShuffleBlocks( - "app0", "exec1", 0, new int[] { 0 }, new int[][] {{ 0, 1 }}); + "app0", "exec1", 0, new long[] { 0 }, new int[][] {{ 0, 1 }}); checkOpenBlocksReceive(fetchShuffleBlocks, blockMarkers); verify(blockResolver, times(1)).getBlockData("app0", "exec1", 0, 0, 0); diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockFetcherSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockFetcherSuite.java index 66633cc7a3595..26a11672b8068 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockFetcherSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockFetcherSuite.java @@ -64,7 +64,7 @@ public void testFetchOne() { BlockFetchingListener listener = fetchBlocks( blocks, blockIds, - new FetchShuffleBlocks("app-id", "exec-id", 0, new int[] { 0 }, new int[][] {{ 0 }}), + new FetchShuffleBlocks("app-id", "exec-id", 0, new long[] { 0 }, new int[][] {{ 0 }}), conf); verify(listener).onBlockFetchSuccess("shuffle_0_0_0", blocks.get("shuffle_0_0_0")); @@ -100,7 +100,7 @@ public void testFetchThreeShuffleBlocks() { BlockFetchingListener listener = fetchBlocks( blocks, blockIds, - new FetchShuffleBlocks("app-id", "exec-id", 0, new int[] { 0 }, new int[][] {{ 0, 1, 2 }}), + new FetchShuffleBlocks("app-id", "exec-id", 0, new long[] { 0 }, new int[][] {{ 0, 1, 2 }}), conf); for (int i = 0; i < 3; i ++) { diff --git a/core/src/main/java/org/apache/spark/shuffle/api/ShuffleExecutorComponents.java b/core/src/main/java/org/apache/spark/shuffle/api/ShuffleExecutorComponents.java index 804119cd06fa6..7f2258daa11d7 100644 --- a/core/src/main/java/org/apache/spark/shuffle/api/ShuffleExecutorComponents.java +++ b/core/src/main/java/org/apache/spark/shuffle/api/ShuffleExecutorComponents.java @@ -40,9 +40,7 @@ public interface ShuffleExecutorComponents { /** * Called once per map task to create a writer that will be responsible for persisting all the * partitioned bytes written by that map task. - * * @param shuffleId Unique identifier for the shuffle the map task is a part of - * @param mapId Within the shuffle, the identifier of the map task * @param mapTaskAttemptId Identifier of the task attempt. Multiple attempts of the same map task * with the same (shuffleId, mapId) pair can be distinguished by the * different values of mapTaskAttemptId. @@ -51,7 +49,6 @@ public interface ShuffleExecutorComponents { */ ShuffleMapOutputWriter createMapOutputWriter( int shuffleId, - int mapId, long mapTaskAttemptId, int numPartitions) throws IOException; @@ -64,14 +61,12 @@ ShuffleMapOutputWriter createMapOutputWriter( * preserving an optimization in the local disk shuffle storage implementation. * * @param shuffleId Unique identifier for the shuffle the map task is a part of - * @param mapId Within the shuffle, the identifier of the map task * @param mapTaskAttemptId Identifier of the task attempt. Multiple attempts of the same map task * with the same (shuffleId, mapId) pair can be distinguished by the * different values of mapTaskAttemptId. */ default Optional createSingleFileMapOutputWriter( int shuffleId, - int mapId, long mapTaskAttemptId) throws IOException { return Optional.empty(); } diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java index f75e932860f90..03e90619516ca 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java @@ -85,7 +85,6 @@ final class BypassMergeSortShuffleWriter extends ShuffleWriter { private final Partitioner partitioner; private final ShuffleWriteMetricsReporter writeMetrics; private final int shuffleId; - private final int mapId; private final long mapTaskAttemptId; private final Serializer serializer; private final ShuffleExecutorComponents shuffleExecutorComponents; @@ -106,7 +105,6 @@ final class BypassMergeSortShuffleWriter extends ShuffleWriter { BypassMergeSortShuffleWriter( BlockManager blockManager, BypassMergeSortShuffleHandle handle, - int mapId, long mapTaskAttemptId, SparkConf conf, ShuffleWriteMetricsReporter writeMetrics, @@ -116,7 +114,6 @@ final class BypassMergeSortShuffleWriter extends ShuffleWriter { this.transferToEnabled = conf.getBoolean("spark.file.transferTo", true); this.blockManager = blockManager; final ShuffleDependency dep = handle.dependency(); - this.mapId = mapId; this.mapTaskAttemptId = mapTaskAttemptId; this.shuffleId = dep.shuffleId(); this.partitioner = dep.partitioner(); @@ -130,11 +127,12 @@ final class BypassMergeSortShuffleWriter extends ShuffleWriter { public void write(Iterator> records) throws IOException { assert (partitionWriters == null); ShuffleMapOutputWriter mapOutputWriter = shuffleExecutorComponents - .createMapOutputWriter(shuffleId, mapId, mapTaskAttemptId, numPartitions); + .createMapOutputWriter(shuffleId, mapTaskAttemptId, numPartitions); try { if (!records.hasNext()) { partitionLengths = mapOutputWriter.commitAllPartitions(); - mapStatus = MapStatus$.MODULE$.apply(blockManager.shuffleServerId(), partitionLengths); + mapStatus = MapStatus$.MODULE$.apply( + blockManager.shuffleServerId(), partitionLengths, mapTaskAttemptId); return; } final SerializerInstance serInstance = serializer.newInstance(); @@ -167,7 +165,8 @@ public void write(Iterator> records) throws IOException { } partitionLengths = writePartitionedData(mapOutputWriter); - mapStatus = MapStatus$.MODULE$.apply(blockManager.shuffleServerId(), partitionLengths); + mapStatus = MapStatus$.MODULE$.apply( + blockManager.shuffleServerId(), partitionLengths, mapTaskAttemptId); } catch (Exception e) { try { mapOutputWriter.abort(e); diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java index 4d11abd36985e..cd82d6abebaa8 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java @@ -78,7 +78,7 @@ public class UnsafeShuffleWriter extends ShuffleWriter { private final ShuffleWriteMetricsReporter writeMetrics; private final ShuffleExecutorComponents shuffleExecutorComponents; private final int shuffleId; - private final int mapId; + private final long mapTaskAttemptId; private final TaskContext taskContext; private final SparkConf sparkConf; private final boolean transferToEnabled; @@ -109,7 +109,6 @@ public UnsafeShuffleWriter( BlockManager blockManager, TaskMemoryManager memoryManager, SerializedShuffleHandle handle, - int mapId, TaskContext taskContext, SparkConf sparkConf, ShuffleWriteMetricsReporter writeMetrics, @@ -123,7 +122,7 @@ public UnsafeShuffleWriter( } this.blockManager = blockManager; this.memoryManager = memoryManager; - this.mapId = mapId; + this.mapTaskAttemptId = taskContext.taskAttemptId(); final ShuffleDependency dep = handle.dependency(); this.shuffleId = dep.shuffleId(); this.serializer = dep.serializer().newInstance(); @@ -228,7 +227,8 @@ void closeAndWriteOutput() throws IOException { } } } - mapStatus = MapStatus$.MODULE$.apply(blockManager.shuffleServerId(), partitionLengths); + mapStatus = MapStatus$.MODULE$.apply( + blockManager.shuffleServerId(), partitionLengths, mapTaskAttemptId); } @VisibleForTesting @@ -266,14 +266,13 @@ private long[] mergeSpills(SpillInfo[] spills) throws IOException { final ShuffleMapOutputWriter mapWriter = shuffleExecutorComponents .createMapOutputWriter( shuffleId, - mapId, taskContext.taskAttemptId(), partitioner.numPartitions()); return mapWriter.commitAllPartitions(); } else if (spills.length == 1) { Optional maybeSingleFileWriter = shuffleExecutorComponents.createSingleFileMapOutputWriter( - shuffleId, mapId, taskContext.taskAttemptId()); + shuffleId, taskContext.taskAttemptId()); if (maybeSingleFileWriter.isPresent()) { // Here, we don't need to perform any metrics updates because the bytes written to this // output file would have already been counted as shuffle bytes written. @@ -300,7 +299,6 @@ private long[] mergeSpillsUsingStandardWriter(SpillInfo[] spills) throws IOExcep final ShuffleMapOutputWriter mapWriter = shuffleExecutorComponents .createMapOutputWriter( shuffleId, - mapId, taskContext.taskAttemptId(), partitioner.numPartitions()); try { diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleExecutorComponents.java b/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleExecutorComponents.java index 47aa2e39fe29b..d3e8c2e4dea1c 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleExecutorComponents.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleExecutorComponents.java @@ -61,7 +61,6 @@ public void initializeExecutor(String appId, String execId) { @Override public ShuffleMapOutputWriter createMapOutputWriter( int shuffleId, - int mapId, long mapTaskAttemptId, int numPartitions) { if (blockResolver == null) { @@ -69,14 +68,13 @@ public ShuffleMapOutputWriter createMapOutputWriter( "Executor components must be initialized before getting writers."); } return new LocalDiskShuffleMapOutputWriter( - shuffleId, mapId, numPartitions, blockResolver, sparkConf); + shuffleId, mapTaskAttemptId, numPartitions, blockResolver, sparkConf); } @Override public Optional createSingleFileMapOutputWriter( int shuffleId, - int mapId, - long mapTaskAttemptId) { + long mapId) { if (blockResolver == null) { throw new IllegalStateException( "Executor components must be initialized before getting writers."); diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriter.java index 444cdc4270ecd..cdf41b4401d7a 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriter.java @@ -48,7 +48,7 @@ public class LocalDiskShuffleMapOutputWriter implements ShuffleMapOutputWriter { LoggerFactory.getLogger(LocalDiskShuffleMapOutputWriter.class); private final int shuffleId; - private final int mapId; + private final long mapTaskAttemptId; private final IndexShuffleBlockResolver blockResolver; private final long[] partitionLengths; private final int bufferSize; @@ -64,18 +64,18 @@ public class LocalDiskShuffleMapOutputWriter implements ShuffleMapOutputWriter { public LocalDiskShuffleMapOutputWriter( int shuffleId, - int mapId, + long mapTaskAttemptId, int numPartitions, IndexShuffleBlockResolver blockResolver, SparkConf sparkConf) { this.shuffleId = shuffleId; - this.mapId = mapId; + this.mapTaskAttemptId = mapTaskAttemptId; this.blockResolver = blockResolver; this.bufferSize = (int) (long) sparkConf.get( package$.MODULE$.SHUFFLE_UNSAFE_FILE_OUTPUT_BUFFER_SIZE()) * 1024; this.partitionLengths = new long[numPartitions]; - this.outputFile = blockResolver.getDataFile(shuffleId, mapId); + this.outputFile = blockResolver.getDataFile(shuffleId, mapTaskAttemptId); this.outputTempFile = null; } @@ -112,7 +112,8 @@ public long[] commitAllPartitions() throws IOException { } cleanUp(); File resolvedTmp = outputTempFile != null && outputTempFile.isFile() ? outputTempFile : null; - blockResolver.writeIndexFileAndCommit(shuffleId, mapId, partitionLengths, resolvedTmp); + blockResolver.writeIndexFileAndCommit( + shuffleId, mapTaskAttemptId, partitionLengths, resolvedTmp); return partitionLengths; } diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskSingleSpillMapOutputWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskSingleSpillMapOutputWriter.java index 6b0a797a61b52..c8b41992a8919 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskSingleSpillMapOutputWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskSingleSpillMapOutputWriter.java @@ -29,12 +29,12 @@ public class LocalDiskSingleSpillMapOutputWriter implements SingleSpillShuffleMapOutputWriter { private final int shuffleId; - private final int mapId; + private final long mapId; private final IndexShuffleBlockResolver blockResolver; public LocalDiskSingleSpillMapOutputWriter( int shuffleId, - int mapId, + long mapId, IndexShuffleBlockResolver blockResolver) { this.shuffleId = shuffleId; this.mapId = mapId; diff --git a/core/src/main/scala/org/apache/spark/Dependency.scala b/core/src/main/scala/org/apache/spark/Dependency.scala index fb051a8c0db8e..f0ac9acd90156 100644 --- a/core/src/main/scala/org/apache/spark/Dependency.scala +++ b/core/src/main/scala/org/apache/spark/Dependency.scala @@ -93,7 +93,7 @@ class ShuffleDependency[K: ClassTag, V: ClassTag, C: ClassTag]( val shuffleId: Int = _rdd.context.newShuffleId() val shuffleHandle: ShuffleHandle = _rdd.context.env.shuffleManager.registerShuffle( - shuffleId, _rdd.partitions.length, this) + shuffleId, this) _rdd.sparkContext.cleaner.foreach(_.registerShuffleForCleanup(this)) } diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index d878fc527791a..4be82d31af764 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -844,7 +844,7 @@ private[spark] object MapOutputTracker extends Logging { statuses: Array[MapStatus]): Iterator[(BlockManagerId, Seq[(BlockId, Long)])] = { assert (statuses != null) val splitsByAddress = new HashMap[BlockManagerId, ListBuffer[(BlockId, Long)]] - for ((status, mapId) <- statuses.iterator.zipWithIndex) { + statuses.foreach { status => if (status == null) { val errorMessage = s"Missing an output location for shuffle $shuffleId" logError(errorMessage) @@ -854,7 +854,7 @@ private[spark] object MapOutputTracker extends Logging { val size = status.getSizeForBlock(part) if (size != 0) { splitsByAddress.getOrElseUpdate(status.location, ListBuffer()) += - ((ShuffleBlockId(shuffleId, mapId, part), size)) + ((ShuffleBlockId(shuffleId, status.mapTaskAttemptId, part), size)) } } } diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index 158a4b7cfa55a..b9693a77d18c5 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -1047,6 +1047,14 @@ package object config { .checkValue(v => v > 0, "The value should be a positive integer.") .createWithDefault(2000) + private[spark] val SHUFFLE_USE_OLD_FETCH_PROTOCOL = + ConfigBuilder("spark.shuffle.useOldFetchProtocol") + .doc("Whether to use the old protocol while doing the shuffle block fetching. " + + "It is only enabled while we need the compatibility in the scenario of new spark " + + "version job fetching shuffle blocks from old version external shuffle service.") + .booleanConf + .createWithDefault(false) + private[spark] val MEMORY_MAP_LIMIT_FOR_TESTS = ConfigBuilder("spark.storage.memoryMapLimitForTests") .internal() diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 894234f70e05a..9fb098486fb2d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1103,7 +1103,16 @@ private[spark] class DAGScheduler( private def submitMissingTasks(stage: Stage, jobId: Int) { logDebug("submitMissingTasks(" + stage + ")") - // First figure out the indexes of partition ids to compute. + // Before find missing partition, do the intermediate state clean work first. + // The operation here can make sure for the intermediate stage, `findMissingPartitions()` + // returns all partitions every time. + stage match { + case sms: ShuffleMapStage if stage.isIndeterminate() => + mapOutputTracker.unregisterAllMapOutput(sms.shuffleDep.shuffleId) + case _ => + } + + // Figure out the indexes of partition ids to compute. val partitionsToCompute: Seq[Int] = stage.findMissingPartitions() // Use the scheduling pool, job group, description, etc. from an ActiveJob associated @@ -1573,7 +1582,7 @@ private[spark] class DAGScheduler( // Note that, if map stage is UNORDERED, we are fine. The shuffle partitioner is // guaranteed to be determinate, so the input data of the reducers will not change // even if the map tasks are re-tried. - if (mapStage.rdd.outputDeterministicLevel == DeterministicLevel.INDETERMINATE) { + if (mapStage.isIndeterminate()) { // It's a little tricky to find all the succeeding stages of `mapStage`, because // each stage only know its parents not children. Here we traverse the stages from // the leaf nodes (the result stages of active jobs), and rollback all the stages @@ -1605,11 +1614,18 @@ private[spark] class DAGScheduler( case mapStage: ShuffleMapStage => val numMissingPartitions = mapStage.findMissingPartitions().length if (numMissingPartitions < mapStage.numTasks) { - // TODO: support to rollback shuffle files. - // Currently the shuffle writing is "first write wins", so we can't re-run a - // shuffle map stage and overwrite existing shuffle files. We have to finish - // SPARK-8029 first. - abortStage(mapStage, generateErrorMessage(mapStage), None) + if (sc.getConf.get(config.SHUFFLE_USE_OLD_FETCH_PROTOCOL)) { + val reason = "A shuffle map stage with indeterminate output was failed " + + "and retried. However, Spark can only do this while using the new " + + "shuffle block fetching protocol. Please check the config " + + "'spark.shuffle.useOldFetchProtocol', see more detail in " + + "SPARK-27665 and SPARK-25341." + abortStage(mapStage, reason, None) + } else { + logInfo(s"The indeterminate stage $mapStage will be resubmitted," + + " the stage self and all indeterminate parent stage will be" + + " rollback and whole stage rerun.") + } } case resultStage: ResultStage if resultStage.activeJob.isDefined => diff --git a/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala b/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala index 64f0a060a247c..f91fe608d80de 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala @@ -43,6 +43,11 @@ private[spark] sealed trait MapStatus { * necessary for correctness, since block fetchers are allowed to skip zero-size blocks. */ def getSizeForBlock(reduceId: Int): Long + + /** + * The unique ID of this shuffle map task, we use taskContext.taskAttemptId to fill this. + */ + def mapTaskAttemptId: Long } @@ -56,11 +61,14 @@ private[spark] object MapStatus { .map(_.conf.get(config.SHUFFLE_MIN_NUM_PARTS_TO_HIGHLY_COMPRESS)) .getOrElse(config.SHUFFLE_MIN_NUM_PARTS_TO_HIGHLY_COMPRESS.defaultValue.get) - def apply(loc: BlockManagerId, uncompressedSizes: Array[Long]): MapStatus = { + def apply( + loc: BlockManagerId, + uncompressedSizes: Array[Long], + mapTaskAttemptId: Long): MapStatus = { if (uncompressedSizes.length > minPartitionsToUseHighlyCompressMapStatus) { - HighlyCompressedMapStatus(loc, uncompressedSizes) + HighlyCompressedMapStatus(loc, uncompressedSizes, mapTaskAttemptId) } else { - new CompressedMapStatus(loc, uncompressedSizes) + new CompressedMapStatus(loc, uncompressedSizes, mapTaskAttemptId) } } @@ -103,13 +111,15 @@ private[spark] object MapStatus { */ private[spark] class CompressedMapStatus( private[this] var loc: BlockManagerId, - private[this] var compressedSizes: Array[Byte]) + private[this] var compressedSizes: Array[Byte], + private[this] var taskAttemptId: Long) extends MapStatus with Externalizable { - protected def this() = this(null, null.asInstanceOf[Array[Byte]]) // For deserialization only + // For deserialization only + protected def this() = this(null, null.asInstanceOf[Array[Byte]], -1) - def this(loc: BlockManagerId, uncompressedSizes: Array[Long]) { - this(loc, uncompressedSizes.map(MapStatus.compressSize)) + def this(loc: BlockManagerId, uncompressedSizes: Array[Long], taskAttemptId: Long) { + this(loc, uncompressedSizes.map(MapStatus.compressSize), taskAttemptId) } override def location: BlockManagerId = loc @@ -118,10 +128,13 @@ private[spark] class CompressedMapStatus( MapStatus.decompressSize(compressedSizes(reduceId)) } + override def mapTaskAttemptId: Long = taskAttemptId + override def writeExternal(out: ObjectOutput): Unit = Utils.tryOrIOException { loc.writeExternal(out) out.writeInt(compressedSizes.length) out.write(compressedSizes) + out.writeLong(taskAttemptId) } override def readExternal(in: ObjectInput): Unit = Utils.tryOrIOException { @@ -129,6 +142,7 @@ private[spark] class CompressedMapStatus( val len = in.readInt() compressedSizes = new Array[Byte](len) in.readFully(compressedSizes) + taskAttemptId = in.readLong() } } @@ -142,20 +156,23 @@ private[spark] class CompressedMapStatus( * @param emptyBlocks a bitmap tracking which blocks are empty * @param avgSize average size of the non-empty and non-huge blocks * @param hugeBlockSizes sizes of huge blocks by their reduceId. + * @param taskAttemptId unique task id for the task */ private[spark] class HighlyCompressedMapStatus private ( private[this] var loc: BlockManagerId, private[this] var numNonEmptyBlocks: Int, private[this] var emptyBlocks: RoaringBitmap, private[this] var avgSize: Long, - private[this] var hugeBlockSizes: scala.collection.Map[Int, Byte]) + private[this] var hugeBlockSizes: scala.collection.Map[Int, Byte], + private[this] var taskAttemptId: Long) extends MapStatus with Externalizable { // loc could be null when the default constructor is called during deserialization - require(loc == null || avgSize > 0 || hugeBlockSizes.size > 0 || numNonEmptyBlocks == 0, + require(loc == null || avgSize > 0 || hugeBlockSizes.size > 0 + || numNonEmptyBlocks == 0 || taskAttemptId > 0, "Average size can only be zero for map stages that produced no output") - protected def this() = this(null, -1, null, -1, null) // For deserialization only + protected def this() = this(null, -1, null, -1, null, -1) // For deserialization only override def location: BlockManagerId = loc @@ -171,6 +188,8 @@ private[spark] class HighlyCompressedMapStatus private ( } } + override def mapTaskAttemptId: Long = taskAttemptId + override def writeExternal(out: ObjectOutput): Unit = Utils.tryOrIOException { loc.writeExternal(out) emptyBlocks.writeExternal(out) @@ -180,6 +199,7 @@ private[spark] class HighlyCompressedMapStatus private ( out.writeInt(kv._1) out.writeByte(kv._2) } + out.writeLong(taskAttemptId) } override def readExternal(in: ObjectInput): Unit = Utils.tryOrIOException { @@ -195,11 +215,15 @@ private[spark] class HighlyCompressedMapStatus private ( hugeBlockSizesImpl(block) = size } hugeBlockSizes = hugeBlockSizesImpl + taskAttemptId = in.readLong() } } private[spark] object HighlyCompressedMapStatus { - def apply(loc: BlockManagerId, uncompressedSizes: Array[Long]): HighlyCompressedMapStatus = { + def apply( + loc: BlockManagerId, + uncompressedSizes: Array[Long], + mapTaskAttemptId: Long): HighlyCompressedMapStatus = { // We must keep track of which blocks are empty so that we don't report a zero-sized // block as being non-empty (or vice-versa) when using the average block size. var i = 0 @@ -240,6 +264,6 @@ private[spark] object HighlyCompressedMapStatus { emptyBlocks.trim() emptyBlocks.runOptimize() new HighlyCompressedMapStatus(loc, numNonEmptyBlocks, emptyBlocks, avgSize, - hugeBlockSizes) + hugeBlockSizes, mapTaskAttemptId) } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala index 710f5eb211dde..06e5d8ab0302a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala @@ -91,7 +91,7 @@ private[spark] class ShuffleMapTask( val rdd = rddAndDep._1 val dep = rddAndDep._2 - dep.shuffleWriterProcessor.write(rdd, dep, partitionId, context, partition) + dep.shuffleWriterProcessor.write(rdd, dep, context, partition) } override def preferredLocations: Seq[TaskLocation] = preferredLocs diff --git a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala index 26cca334d3bd5..135d0036ef734 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala @@ -21,7 +21,7 @@ import scala.collection.mutable.HashSet import org.apache.spark.executor.TaskMetrics import org.apache.spark.internal.Logging -import org.apache.spark.rdd.RDD +import org.apache.spark.rdd.{DeterministicLevel, RDD} import org.apache.spark.util.CallSite /** @@ -116,4 +116,8 @@ private[scheduler] abstract class Stage( /** Returns the sequence of partition ids that are missing (i.e. needs to be computed). */ def findMissingPartitions(): Seq[Int] + + def isIndeterminate(): Boolean = { + rdd.outputDeterministicLevel == DeterministicLevel.INDETERMINATE + } } diff --git a/core/src/main/scala/org/apache/spark/shuffle/BaseShuffleHandle.scala b/core/src/main/scala/org/apache/spark/shuffle/BaseShuffleHandle.scala index 04e4cf88d7063..6fe183c078089 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/BaseShuffleHandle.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/BaseShuffleHandle.scala @@ -24,6 +24,5 @@ import org.apache.spark.ShuffleDependency */ private[spark] class BaseShuffleHandle[K, V, C]( shuffleId: Int, - val numMaps: Int, val dependency: ShuffleDependency[K, V, C]) extends ShuffleHandle(shuffleId) diff --git a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala index d3f1c7ec1bbee..332164a7be3e7 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala @@ -51,18 +51,18 @@ private[spark] class IndexShuffleBlockResolver( private val transportConf = SparkTransportConf.fromSparkConf(conf, "shuffle") - def getDataFile(shuffleId: Int, mapId: Int): File = { + def getDataFile(shuffleId: Int, mapId: Long): File = { blockManager.diskBlockManager.getFile(ShuffleDataBlockId(shuffleId, mapId, NOOP_REDUCE_ID)) } - private def getIndexFile(shuffleId: Int, mapId: Int): File = { + private def getIndexFile(shuffleId: Int, mapId: Long): File = { blockManager.diskBlockManager.getFile(ShuffleIndexBlockId(shuffleId, mapId, NOOP_REDUCE_ID)) } /** * Remove data file and index file that contain the output data from one map. */ - def removeDataByMap(shuffleId: Int, mapId: Int): Unit = { + def removeDataByMap(shuffleId: Int, mapId: Long): Unit = { var file = getDataFile(shuffleId, mapId) if (file.exists()) { if (!file.delete()) { @@ -135,7 +135,7 @@ private[spark] class IndexShuffleBlockResolver( */ def writeIndexFileAndCommit( shuffleId: Int, - mapId: Int, + mapId: Long, lengths: Array[Long], dataTmp: File): Unit = { val indexFile = getIndexFile(shuffleId, mapId) diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala index 18a743fbfa6fc..a2fd2da6fe97a 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala @@ -34,13 +34,11 @@ private[spark] trait ShuffleManager { */ def registerShuffle[K, V, C]( shuffleId: Int, - numMaps: Int, dependency: ShuffleDependency[K, V, C]): ShuffleHandle /** Get a writer for a given partition. Called on executors by map tasks. */ def getWriter[K, V]( handle: ShuffleHandle, - mapId: Int, context: TaskContext, metrics: ShuffleWriteMetricsReporter): ShuffleWriter[K, V] diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriteProcessor.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriteProcessor.scala index 5b0c7e9f2b0b4..68b589316e9e2 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriteProcessor.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriteProcessor.scala @@ -44,7 +44,6 @@ private[spark] class ShuffleWriteProcessor extends Serializable with Logging { def write( rdd: RDD[_], dep: ShuffleDependency[_, _, _], - partitionId: Int, context: TaskContext, partition: Partition): MapStatus = { var writer: ShuffleWriter[Any, Any] = null @@ -52,7 +51,6 @@ private[spark] class ShuffleWriteProcessor extends Serializable with Logging { val manager = SparkEnv.get.shuffleManager writer = manager.getWriter[Any, Any]( dep.shuffleHandle, - partitionId, context, createMetricsReporter(context)) writer.write( diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala index cbdc2c886dd9f..32b0536abba3a 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala @@ -19,6 +19,8 @@ package org.apache.spark.shuffle.sort import java.util.concurrent.ConcurrentHashMap +import scala.collection.mutable.ArrayBuffer + import org.apache.spark._ import org.apache.spark.internal.{config, Logging} import org.apache.spark.shuffle._ @@ -79,9 +81,9 @@ private[spark] class SortShuffleManager(conf: SparkConf) extends ShuffleManager } /** - * A mapping from shuffle ids to the number of mappers producing output for those shuffles. + * A mapping from shuffle ids to the task ids of mappers producing output for those shuffles. */ - private[this] val numMapsForShuffle = new ConcurrentHashMap[Int, Int]() + private[this] val taskIdMapsForShuffle = new ConcurrentHashMap[Int, ArrayBuffer[Long]]() private lazy val shuffleExecutorComponents = loadShuffleExecutorComponents(conf) @@ -92,7 +94,6 @@ private[spark] class SortShuffleManager(conf: SparkConf) extends ShuffleManager */ override def registerShuffle[K, V, C]( shuffleId: Int, - numMaps: Int, dependency: ShuffleDependency[K, V, C]): ShuffleHandle = { if (SortShuffleWriter.shouldBypassMergeSort(conf, dependency)) { // If there are fewer than spark.shuffle.sort.bypassMergeThreshold partitions and we don't @@ -101,14 +102,14 @@ private[spark] class SortShuffleManager(conf: SparkConf) extends ShuffleManager // together the spilled files, which would happen with the normal code path. The downside is // having multiple files open at a time and thus more memory allocated to buffers. new BypassMergeSortShuffleHandle[K, V]( - shuffleId, numMaps, dependency.asInstanceOf[ShuffleDependency[K, V, V]]) + shuffleId, dependency.asInstanceOf[ShuffleDependency[K, V, V]]) } else if (SortShuffleManager.canUseSerializedShuffle(dependency)) { // Otherwise, try to buffer map outputs in a serialized form, since this is more efficient: new SerializedShuffleHandle[K, V]( - shuffleId, numMaps, dependency.asInstanceOf[ShuffleDependency[K, V, V]]) + shuffleId, dependency.asInstanceOf[ShuffleDependency[K, V, V]]) } else { // Otherwise, buffer map outputs in a deserialized form: - new BaseShuffleHandle(shuffleId, numMaps, dependency) + new BaseShuffleHandle(shuffleId, dependency) } } @@ -130,11 +131,12 @@ private[spark] class SortShuffleManager(conf: SparkConf) extends ShuffleManager /** Get a writer for a given partition. Called on executors by map tasks. */ override def getWriter[K, V]( handle: ShuffleHandle, - mapId: Int, context: TaskContext, metrics: ShuffleWriteMetricsReporter): ShuffleWriter[K, V] = { - numMapsForShuffle.putIfAbsent( - handle.shuffleId, handle.asInstanceOf[BaseShuffleHandle[_, _, _]].numMaps) + taskIdMapsForShuffle.synchronized { + taskIdMapsForShuffle.putIfAbsent(handle.shuffleId, ArrayBuffer.empty[Long]) + taskIdMapsForShuffle.get(handle.shuffleId).append(context.taskAttemptId()) + } val env = SparkEnv.get handle match { case unsafeShuffleHandle: SerializedShuffleHandle[K @unchecked, V @unchecked] => @@ -142,7 +144,6 @@ private[spark] class SortShuffleManager(conf: SparkConf) extends ShuffleManager env.blockManager, context.taskMemoryManager(), unsafeShuffleHandle, - mapId, context, env.conf, metrics, @@ -151,21 +152,20 @@ private[spark] class SortShuffleManager(conf: SparkConf) extends ShuffleManager new BypassMergeSortShuffleWriter( env.blockManager, bypassMergeSortHandle, - mapId, context.taskAttemptId(), env.conf, metrics, shuffleExecutorComponents) case other: BaseShuffleHandle[K @unchecked, V @unchecked, _] => new SortShuffleWriter( - shuffleBlockResolver, other, mapId, context, shuffleExecutorComponents) + shuffleBlockResolver, other, context, shuffleExecutorComponents) } } /** Remove a shuffle's metadata from the ShuffleManager. */ override def unregisterShuffle(shuffleId: Int): Boolean = { - Option(numMapsForShuffle.remove(shuffleId)).foreach { numMaps => - (0 until numMaps).foreach { mapId => + Option(taskIdMapsForShuffle.remove(shuffleId)).foreach { mapIds => + mapIds.foreach { mapId => shuffleBlockResolver.removeDataByMap(shuffleId, mapId) } } @@ -231,9 +231,8 @@ private[spark] object SortShuffleManager extends Logging { */ private[spark] class SerializedShuffleHandle[K, V]( shuffleId: Int, - numMaps: Int, dependency: ShuffleDependency[K, V, V]) - extends BaseShuffleHandle(shuffleId, numMaps, dependency) { + extends BaseShuffleHandle(shuffleId, dependency) { } /** @@ -242,7 +241,6 @@ private[spark] class SerializedShuffleHandle[K, V]( */ private[spark] class BypassMergeSortShuffleHandle[K, V]( shuffleId: Int, - numMaps: Int, dependency: ShuffleDependency[K, V, V]) - extends BaseShuffleHandle(shuffleId, numMaps, dependency) { + extends BaseShuffleHandle(shuffleId, dependency) { } diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala index a781b16252432..264cc2aa9f901 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala @@ -27,7 +27,6 @@ import org.apache.spark.util.collection.ExternalSorter private[spark] class SortShuffleWriter[K, V, C]( shuffleBlockResolver: IndexShuffleBlockResolver, handle: BaseShuffleHandle[K, V, C], - mapId: Int, context: TaskContext, shuffleExecutorComponents: ShuffleExecutorComponents) extends ShuffleWriter[K, V] with Logging { @@ -64,11 +63,12 @@ private[spark] class SortShuffleWriter[K, V, C]( // Don't bother including the time to open the merged output file in the shuffle write time, // because it just opens a single file, so is typically too fast to measure accurately // (see SPARK-3570). + val mapTaskAttemptId = context.taskAttemptId() val mapOutputWriter = shuffleExecutorComponents.createMapOutputWriter( - dep.shuffleId, mapId, context.taskAttemptId(), dep.partitioner.numPartitions) - sorter.writePartitionedMapOutput(dep.shuffleId, mapId, mapOutputWriter) + dep.shuffleId, mapTaskAttemptId, dep.partitioner.numPartitions) + sorter.writePartitionedMapOutput(dep.shuffleId, mapTaskAttemptId, mapOutputWriter) val partitionLengths = mapOutputWriter.commitAllPartitions() - mapStatus = MapStatus(blockManager.shuffleServerId, partitionLengths) + mapStatus = MapStatus(blockManager.shuffleServerId, partitionLengths, mapTaskAttemptId) } /** Close this writer, passing along whether the map completed */ diff --git a/core/src/main/scala/org/apache/spark/storage/BlockId.scala b/core/src/main/scala/org/apache/spark/storage/BlockId.scala index 7ac2c71c18eb3..9c5b7f64e7abe 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockId.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockId.scala @@ -52,17 +52,17 @@ case class RDDBlockId(rddId: Int, splitIndex: Int) extends BlockId { // Format of the shuffle block ids (including data and index) should be kept in sync with // org.apache.spark.network.shuffle.ExternalShuffleBlockResolver#getBlockData(). @DeveloperApi -case class ShuffleBlockId(shuffleId: Int, mapId: Int, reduceId: Int) extends BlockId { +case class ShuffleBlockId(shuffleId: Int, mapId: Long, reduceId: Int) extends BlockId { override def name: String = "shuffle_" + shuffleId + "_" + mapId + "_" + reduceId } @DeveloperApi -case class ShuffleDataBlockId(shuffleId: Int, mapId: Int, reduceId: Int) extends BlockId { +case class ShuffleDataBlockId(shuffleId: Int, mapId: Long, reduceId: Int) extends BlockId { override def name: String = "shuffle_" + shuffleId + "_" + mapId + "_" + reduceId + ".data" } @DeveloperApi -case class ShuffleIndexBlockId(shuffleId: Int, mapId: Int, reduceId: Int) extends BlockId { +case class ShuffleIndexBlockId(shuffleId: Int, mapId: Long, reduceId: Int) extends BlockId { override def name: String = "shuffle_" + shuffleId + "_" + mapId + "_" + reduceId + ".index" } @@ -117,11 +117,11 @@ object BlockId { case RDD(rddId, splitIndex) => RDDBlockId(rddId.toInt, splitIndex.toInt) case SHUFFLE(shuffleId, mapId, reduceId) => - ShuffleBlockId(shuffleId.toInt, mapId.toInt, reduceId.toInt) + ShuffleBlockId(shuffleId.toInt, mapId.toLong, reduceId.toInt) case SHUFFLE_DATA(shuffleId, mapId, reduceId) => - ShuffleDataBlockId(shuffleId.toInt, mapId.toInt, reduceId.toInt) + ShuffleDataBlockId(shuffleId.toInt, mapId.toLong, reduceId.toInt) case SHUFFLE_INDEX(shuffleId, mapId, reduceId) => - ShuffleIndexBlockId(shuffleId.toInt, mapId.toInt, reduceId.toInt) + ShuffleIndexBlockId(shuffleId.toInt, mapId.toLong, reduceId.toInt) case BROADCAST(broadcastId, field) => BroadcastBlockId(broadcastId.toLong, field.stripPrefix("_")) case TASKRESULT(taskId) => diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala index 7a822e137e556..6fecfbaca8416 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala @@ -727,7 +727,7 @@ private[spark] class ExternalSorter[K, V, C]( */ def writePartitionedMapOutput( shuffleId: Int, - mapId: Int, + mapId: Long, mapOutputWriter: ShuffleMapOutputWriter): Unit = { var nextPartitionId = 0 if (spills.isEmpty) { diff --git a/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java b/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java index 1022111897a49..ca5958658c2f8 100644 --- a/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java +++ b/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java @@ -135,7 +135,7 @@ public void setUp() throws IOException { ); }); - when(shuffleBlockResolver.getDataFile(anyInt(), anyInt())).thenReturn(mergedOutputFile); + when(shuffleBlockResolver.getDataFile(anyInt(), anyLong())).thenReturn(mergedOutputFile); Answer renameTempAnswer = invocationOnMock -> { partitionSizesInMergedFile = (long[]) invocationOnMock.getArguments()[2]; @@ -157,7 +157,7 @@ public void setUp() throws IOException { doAnswer(renameTempAnswer) .when(shuffleBlockResolver) - .writeIndexFileAndCommit(anyInt(), anyInt(), any(long[].class), eq(null)); + .writeIndexFileAndCommit(anyInt(), anyLong(), any(long[].class), any(File.class)); when(diskBlockManager.createTempShuffleBlock()).thenAnswer(invocationOnMock -> { TempShuffleBlockId blockId = new TempShuffleBlockId(UUID.randomUUID()); @@ -176,9 +176,8 @@ private UnsafeShuffleWriter createWriter(boolean transferToEnabled) { conf.set("spark.file.transferTo", String.valueOf(transferToEnabled)); return new UnsafeShuffleWriter( blockManager, - taskMemoryManager, - new SerializedShuffleHandle<>(0, 1, shuffleDep), - 0, // map id + taskMemoryManager, + new SerializedShuffleHandle<>(0, shuffleDep), taskContext, conf, taskContext.taskMetrics().shuffleWriteMetrics(), @@ -536,8 +535,7 @@ public void testPeakMemoryUsed() throws Exception { final UnsafeShuffleWriter writer = new UnsafeShuffleWriter( blockManager, taskMemoryManager, - new SerializedShuffleHandle<>(0, 1, shuffleDep), - 0, // map id + new SerializedShuffleHandle<>(0, shuffleDep), taskContext, conf, taskContext.taskMetrics().shuffleWriteMetrics(), diff --git a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala index d86975964b558..06602563693a1 100644 --- a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala @@ -64,13 +64,13 @@ class MapOutputTrackerSuite extends SparkFunSuite { val size1000 = MapStatus.decompressSize(MapStatus.compressSize(1000L)) val size10000 = MapStatus.decompressSize(MapStatus.compressSize(10000L)) tracker.registerMapOutput(10, 0, MapStatus(BlockManagerId("a", "hostA", 1000), - Array(1000L, 10000L))) + Array(1000L, 10000L), 5)) tracker.registerMapOutput(10, 1, MapStatus(BlockManagerId("b", "hostB", 1000), - Array(10000L, 1000L))) + Array(10000L, 1000L), 6)) val statuses = tracker.getMapSizesByExecutorId(10, 0) assert(statuses.toSet === - Seq((BlockManagerId("a", "hostA", 1000), ArrayBuffer((ShuffleBlockId(10, 0, 0), size1000))), - (BlockManagerId("b", "hostB", 1000), ArrayBuffer((ShuffleBlockId(10, 1, 0), size10000)))) + Seq((BlockManagerId("a", "hostA", 1000), ArrayBuffer((ShuffleBlockId(10, 5, 0), size1000))), + (BlockManagerId("b", "hostB", 1000), ArrayBuffer((ShuffleBlockId(10, 6, 0), size10000)))) .toSet) assert(0 == tracker.getNumCachedSerializedBroadcast) tracker.stop() @@ -86,9 +86,9 @@ class MapOutputTrackerSuite extends SparkFunSuite { val compressedSize1000 = MapStatus.compressSize(1000L) val compressedSize10000 = MapStatus.compressSize(10000L) tracker.registerMapOutput(10, 0, MapStatus(BlockManagerId("a", "hostA", 1000), - Array(compressedSize1000, compressedSize10000))) + Array(compressedSize1000, compressedSize10000), 5)) tracker.registerMapOutput(10, 1, MapStatus(BlockManagerId("b", "hostB", 1000), - Array(compressedSize10000, compressedSize1000))) + Array(compressedSize10000, compressedSize1000), 6)) assert(tracker.containsShuffle(10)) assert(tracker.getMapSizesByExecutorId(10, 0).nonEmpty) assert(0 == tracker.getNumCachedSerializedBroadcast) @@ -109,9 +109,9 @@ class MapOutputTrackerSuite extends SparkFunSuite { val compressedSize1000 = MapStatus.compressSize(1000L) val compressedSize10000 = MapStatus.compressSize(10000L) tracker.registerMapOutput(10, 0, MapStatus(BlockManagerId("a", "hostA", 1000), - Array(compressedSize1000, compressedSize1000, compressedSize1000))) + Array(compressedSize1000, compressedSize1000, compressedSize1000), 5)) tracker.registerMapOutput(10, 1, MapStatus(BlockManagerId("b", "hostB", 1000), - Array(compressedSize10000, compressedSize1000, compressedSize1000))) + Array(compressedSize10000, compressedSize1000, compressedSize1000), 6)) assert(0 == tracker.getNumCachedSerializedBroadcast) // As if we had two simultaneous fetch failures @@ -147,10 +147,10 @@ class MapOutputTrackerSuite extends SparkFunSuite { val size1000 = MapStatus.decompressSize(MapStatus.compressSize(1000L)) masterTracker.registerMapOutput(10, 0, MapStatus( - BlockManagerId("a", "hostA", 1000), Array(1000L))) + BlockManagerId("a", "hostA", 1000), Array(1000L), 5)) slaveTracker.updateEpoch(masterTracker.getEpoch) assert(slaveTracker.getMapSizesByExecutorId(10, 0).toSeq === - Seq((BlockManagerId("a", "hostA", 1000), ArrayBuffer((ShuffleBlockId(10, 0, 0), size1000))))) + Seq((BlockManagerId("a", "hostA", 1000), ArrayBuffer((ShuffleBlockId(10, 5, 0), size1000))))) assert(0 == masterTracker.getNumCachedSerializedBroadcast) val masterTrackerEpochBeforeLossOfMapOutput = masterTracker.getEpoch @@ -184,7 +184,7 @@ class MapOutputTrackerSuite extends SparkFunSuite { // Message size should be ~123B, and no exception should be thrown masterTracker.registerShuffle(10, 1) masterTracker.registerMapOutput(10, 0, MapStatus( - BlockManagerId("88", "mph", 1000), Array.fill[Long](10)(0))) + BlockManagerId("88", "mph", 1000), Array.fill[Long](10)(0), 5)) val senderAddress = RpcAddress("localhost", 12345) val rpcCallContext = mock(classOf[RpcCallContext]) when(rpcCallContext.senderAddress).thenReturn(senderAddress) @@ -218,11 +218,11 @@ class MapOutputTrackerSuite extends SparkFunSuite { // on hostB with output size 3 tracker.registerShuffle(10, 3) tracker.registerMapOutput(10, 0, MapStatus(BlockManagerId("a", "hostA", 1000), - Array(2L))) + Array(2L), 5)) tracker.registerMapOutput(10, 1, MapStatus(BlockManagerId("a", "hostA", 1000), - Array(2L))) + Array(2L), 6)) tracker.registerMapOutput(10, 2, MapStatus(BlockManagerId("b", "hostB", 1000), - Array(3L))) + Array(3L), 7)) // When the threshold is 50%, only host A should be returned as a preferred location // as it has 4 out of 7 bytes of output. @@ -262,7 +262,7 @@ class MapOutputTrackerSuite extends SparkFunSuite { masterTracker.registerShuffle(20, 100) (0 until 100).foreach { i => masterTracker.registerMapOutput(20, i, new CompressedMapStatus( - BlockManagerId("999", "mps", 1000), Array.fill[Long](4000000)(0))) + BlockManagerId("999", "mps", 1000), Array.fill[Long](4000000)(0), 5)) } val senderAddress = RpcAddress("localhost", 12345) val rpcCallContext = mock(classOf[RpcCallContext]) @@ -311,16 +311,16 @@ class MapOutputTrackerSuite extends SparkFunSuite { val size1000 = MapStatus.decompressSize(MapStatus.compressSize(1000L)) val size10000 = MapStatus.decompressSize(MapStatus.compressSize(10000L)) tracker.registerMapOutput(10, 0, MapStatus(BlockManagerId("a", "hostA", 1000), - Array(size0, size1000, size0, size10000))) + Array(size0, size1000, size0, size10000), 5)) tracker.registerMapOutput(10, 1, MapStatus(BlockManagerId("b", "hostB", 1000), - Array(size10000, size0, size1000, size0))) + Array(size10000, size0, size1000, size0), 6)) assert(tracker.containsShuffle(10)) assert(tracker.getMapSizesByExecutorId(10, 0, 4).toSeq === Seq( (BlockManagerId("a", "hostA", 1000), - Seq((ShuffleBlockId(10, 0, 1), size1000), (ShuffleBlockId(10, 0, 3), size10000))), + Seq((ShuffleBlockId(10, 5, 1), size1000), (ShuffleBlockId(10, 5, 3), size10000))), (BlockManagerId("b", "hostB", 1000), - Seq((ShuffleBlockId(10, 1, 0), size10000), (ShuffleBlockId(10, 1, 2), size1000))) + Seq((ShuffleBlockId(10, 6, 0), size10000), (ShuffleBlockId(10, 6, 2), size1000))) ) ) diff --git a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala index 923c9c90447fd..5dc62f78449fe 100644 --- a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala +++ b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala @@ -360,14 +360,14 @@ abstract class ShuffleSuite extends SparkFunSuite with Matchers with LocalSparkC val metricsSystem = sc.env.metricsSystem val shuffleMapRdd = new MyRDD(sc, 1, Nil) val shuffleDep = new ShuffleDependency(shuffleMapRdd, new HashPartitioner(1)) - val shuffleHandle = manager.registerShuffle(0, 1, shuffleDep) + val shuffleHandle = manager.registerShuffle(0, shuffleDep) mapTrackerMaster.registerShuffle(0, 1) // first attempt -- its successful val context1 = new TaskContextImpl(0, 0, 0, 0L, 0, taskMemoryManager, new Properties, metricsSystem) val writer1 = manager.getWriter[Int, Int]( - shuffleHandle, 0, context1, context1.taskMetrics.shuffleWriteMetrics) + shuffleHandle, context1, context1.taskMetrics.shuffleWriteMetrics) val data1 = (1 to 10).map { x => x -> x} // second attempt -- also successful. We'll write out different data, @@ -376,7 +376,7 @@ abstract class ShuffleSuite extends SparkFunSuite with Matchers with LocalSparkC val context2 = new TaskContextImpl(0, 0, 0, 1L, 0, taskMemoryManager, new Properties, metricsSystem) val writer2 = manager.getWriter[Int, Int]( - shuffleHandle, 0, context2, context2.taskMetrics.shuffleWriteMetrics) + shuffleHandle, context2, context2.taskMetrics.shuffleWriteMetrics) val data2 = (11 to 20).map { x => x -> x} // interleave writes of both attempts -- we want to test that both attempts can occur diff --git a/core/src/test/scala/org/apache/spark/memory/MemoryTestingUtils.scala b/core/src/test/scala/org/apache/spark/memory/MemoryTestingUtils.scala index dcf89e4f75acf..72708bc41a533 100644 --- a/core/src/test/scala/org/apache/spark/memory/MemoryTestingUtils.scala +++ b/core/src/test/scala/org/apache/spark/memory/MemoryTestingUtils.scala @@ -25,13 +25,13 @@ import org.apache.spark.{SparkEnv, TaskContext, TaskContextImpl} * Helper methods for mocking out memory-management-related classes in tests. */ object MemoryTestingUtils { - def fakeTaskContext(env: SparkEnv): TaskContext = { + def fakeTaskContext(env: SparkEnv, taskAttemptId: Long = 0): TaskContext = { val taskMemoryManager = new TaskMemoryManager(env.memoryManager, 0) new TaskContextImpl( stageId = 0, stageAttemptNumber = 0, partitionId = 0, - taskAttemptId = 0, + taskAttemptId = taskAttemptId, attemptNumber = 0, taskMemoryManager = taskMemoryManager, localProperties = new Properties, diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 7cb7eceec615b..cded7061d566a 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -487,18 +487,18 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // map stage1 completes successfully, with one task on each executor complete(taskSets(0), Seq( (Success, - MapStatus(BlockManagerId("exec-hostA1", "hostA", 12345), Array.fill[Long](1)(2))), + MapStatus(BlockManagerId("exec-hostA1", "hostA", 12345), Array.fill[Long](1)(2), 5)), (Success, - MapStatus(BlockManagerId("exec-hostA2", "hostA", 12345), Array.fill[Long](1)(2))), - (Success, makeMapStatus("hostB", 1)) + MapStatus(BlockManagerId("exec-hostA2", "hostA", 12345), Array.fill[Long](1)(2), 6)), + (Success, makeMapStatus("hostB", 1, taskAttemptId = 7)) )) // map stage2 completes successfully, with one task on each executor complete(taskSets(1), Seq( (Success, - MapStatus(BlockManagerId("exec-hostA1", "hostA", 12345), Array.fill[Long](1)(2))), + MapStatus(BlockManagerId("exec-hostA1", "hostA", 12345), Array.fill[Long](1)(2), 8)), (Success, - MapStatus(BlockManagerId("exec-hostA2", "hostA", 12345), Array.fill[Long](1)(2))), - (Success, makeMapStatus("hostB", 1)) + MapStatus(BlockManagerId("exec-hostA2", "hostA", 12345), Array.fill[Long](1)(2), 9)), + (Success, makeMapStatus("hostB", 1, taskAttemptId = 10)) )) // make sure our test setup is correct val initialMapStatus1 = mapOutputTracker.shuffleStatuses(firstShuffleId).mapStatuses @@ -506,12 +506,14 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi assert(initialMapStatus1.count(_ != null) === 3) assert(initialMapStatus1.map{_.location.executorId}.toSet === Set("exec-hostA1", "exec-hostA2", "exec-hostB")) + assert(initialMapStatus1.map{_.mapTaskAttemptId}.toSet === Set(5, 6, 7)) val initialMapStatus2 = mapOutputTracker.shuffleStatuses(secondShuffleId).mapStatuses // val initialMapStatus1 = mapOutputTracker.mapStatuses.get(0).get assert(initialMapStatus2.count(_ != null) === 3) assert(initialMapStatus2.map{_.location.executorId}.toSet === Set("exec-hostA1", "exec-hostA2", "exec-hostB")) + assert(initialMapStatus2.map{_.mapTaskAttemptId}.toSet === Set(8, 9, 10)) // reduce stage fails with a fetch failure from one host complete(taskSets(2), Seq( @@ -2745,7 +2747,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi assert(countSubmittedMapStageAttempts() === 2) } - test("SPARK-23207: retry all the succeeding stages when the map stage is indeterminate") { + private def constructIndeterminateStageFetchFailed(): (Int, Int) = { val shuffleMapRdd1 = new MyRDD(sc, 2, Nil, indeterminate = true) val shuffleDep1 = new ShuffleDependency(shuffleMapRdd1, new HashPartitioner(2)) @@ -2775,12 +2777,138 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi taskSets(2).tasks(0), FetchFailed(makeBlockManagerId("hostC"), shuffleId2, 0, 0, "ignored"), null)) + (shuffleId1, shuffleId2) + } + + test("SPARK-25341: abort stage while using old fetch protocol") { + // reset the test context with using old fetch protocol + afterEach() + val conf = new SparkConf() + conf.set(config.SHUFFLE_USE_OLD_FETCH_PROTOCOL.key, "true") + init(conf) + // Construct the scenario of indeterminate stage fetch failed. + constructIndeterminateStageFetchFailed() + // The job should fail because Spark can't rollback the shuffle map stage while + // using old protocol. + assert(failure != null && failure.getMessage.contains( + "Spark can only do this while using the new shuffle block fetching protocol")) + } + + test("SPARK-25341: retry all the succeeding stages when the map stage is indeterminate") { + val (shuffleId1, shuffleId2) = constructIndeterminateStageFetchFailed() + + // Check status for all failedStages + val failedStages = scheduler.failedStages.toSeq + assert(failedStages.map(_.id) == Seq(1, 2)) + // Shuffle blocks of "hostC" is lost, so first task of the `shuffleMapRdd2` needs to retry. + assert(failedStages.collect { + case stage: ShuffleMapStage if stage.shuffleDep.shuffleId == shuffleId2 => stage + }.head.findMissingPartitions() == Seq(0)) + // The result stage is still waiting for its 2 tasks to complete + assert(failedStages.collect { + case stage: ResultStage => stage + }.head.findMissingPartitions() == Seq(0, 1)) + + scheduler.resubmitFailedStages() + + // The first task of the `shuffleMapRdd2` failed with fetch failure + runEvent(makeCompletionEvent( + taskSets(3).tasks(0), + FetchFailed(makeBlockManagerId("hostA"), shuffleId1, 0, 0, "ignored"), + null)) + + val newFailedStages = scheduler.failedStages.toSeq + assert(newFailedStages.map(_.id) == Seq(0, 1)) + + scheduler.resubmitFailedStages() + + // First shuffle map stage resubmitted and reran all tasks. + assert(taskSets(4).stageId == 0) + assert(taskSets(4).stageAttemptId == 1) + assert(taskSets(4).tasks.length == 2) + + // Finish all stage. + complete(taskSets(4), Seq( + (Success, makeMapStatus("hostA", 2)), + (Success, makeMapStatus("hostB", 2)))) + assert(mapOutputTracker.findMissingPartitions(shuffleId1) === Some(Seq.empty)) + + complete(taskSets(5), Seq( + (Success, makeMapStatus("hostC", 2)), + (Success, makeMapStatus("hostD", 2)))) + assert(mapOutputTracker.findMissingPartitions(shuffleId2) === Some(Seq.empty)) + + complete(taskSets(6), Seq((Success, 11), (Success, 12))) + + // Job successful ended. + assert(results === Map(0 -> 11, 1 -> 12)) + results.clear() + assertDataStructuresEmpty() + } + + test("SPARK-25341: continuous indeterminate stage roll back") { + // shuffleMapRdd1/2/3 are all indeterminate. + val shuffleMapRdd1 = new MyRDD(sc, 2, Nil, indeterminate = true) + val shuffleDep1 = new ShuffleDependency(shuffleMapRdd1, new HashPartitioner(2)) + val shuffleId1 = shuffleDep1.shuffleId + + val shuffleMapRdd2 = new MyRDD( + sc, 2, List(shuffleDep1), tracker = mapOutputTracker, indeterminate = true) + val shuffleDep2 = new ShuffleDependency(shuffleMapRdd2, new HashPartitioner(2)) + val shuffleId2 = shuffleDep2.shuffleId - // The second shuffle map stage need to rerun, the job will abort for the indeterminate - // stage rerun. - // TODO: After we support re-generate shuffle file(SPARK-25341), this test will be extended. - assert(failure != null && failure.getMessage - .contains("Spark cannot rollback the ShuffleMapStage 1")) + val shuffleMapRdd3 = new MyRDD( + sc, 2, List(shuffleDep2), tracker = mapOutputTracker, indeterminate = true) + val shuffleDep3 = new ShuffleDependency(shuffleMapRdd3, new HashPartitioner(2)) + val shuffleId3 = shuffleDep3.shuffleId + val finalRdd = new MyRDD(sc, 2, List(shuffleDep3), tracker = mapOutputTracker) + + submit(finalRdd, Array(0, 1), properties = new Properties()) + + // Finish the first 2 shuffle map stages. + complete(taskSets(0), Seq( + (Success, makeMapStatus("hostA", 2)), + (Success, makeMapStatus("hostB", 2)))) + assert(mapOutputTracker.findMissingPartitions(shuffleId1) === Some(Seq.empty)) + + complete(taskSets(1), Seq( + (Success, makeMapStatus("hostB", 2)), + (Success, makeMapStatus("hostD", 2)))) + assert(mapOutputTracker.findMissingPartitions(shuffleId2) === Some(Seq.empty)) + + // Executor lost on hostB, both of stage 0 and 1 should be reran. + runEvent(makeCompletionEvent( + taskSets(2).tasks(0), + FetchFailed(makeBlockManagerId("hostB"), shuffleId2, 0, 0, "ignored"), + null)) + mapOutputTracker.removeOutputsOnHost("hostB") + + assert(scheduler.failedStages.toSeq.map(_.id) == Seq(1, 2)) + scheduler.resubmitFailedStages() + + def checkAndCompleteRetryStage( + taskSetIndex: Int, + stageId: Int, + shuffleId: Int): Unit = { + assert(taskSets(taskSetIndex).stageId == stageId) + assert(taskSets(taskSetIndex).stageAttemptId == 1) + assert(taskSets(taskSetIndex).tasks.length == 2) + complete(taskSets(taskSetIndex), Seq( + (Success, makeMapStatus("hostA", 2)), + (Success, makeMapStatus("hostB", 2)))) + assert(mapOutputTracker.findMissingPartitions(shuffleId) === Some(Seq.empty)) + } + + // Check all indeterminate stage roll back. + checkAndCompleteRetryStage(3, 0, shuffleId1) + checkAndCompleteRetryStage(4, 1, shuffleId2) + checkAndCompleteRetryStage(5, 2, shuffleId3) + + // Result stage success, all job ended. + complete(taskSets(6), Seq((Success, 11), (Success, 12))) + assert(results === Map(0 -> 11, 1 -> 12)) + results.clear() + assertDataStructuresEmpty() } test("SPARK-29042: Sampled RDD with unordered input should be indeterminate") { @@ -2903,33 +3031,6 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi assert(latch.await(10, TimeUnit.SECONDS)) } - test("SPARK-28699: abort stage if parent stage is indeterminate stage") { - val shuffleMapRdd = new MyRDD(sc, 2, Nil, indeterminate = true) - - val shuffleDep = new ShuffleDependency(shuffleMapRdd, new HashPartitioner(2)) - val shuffleId = shuffleDep.shuffleId - val finalRdd = new MyRDD(sc, 2, List(shuffleDep), tracker = mapOutputTracker) - - submit(finalRdd, Array(0, 1)) - - // Finish the first shuffle map stage. - complete(taskSets(0), Seq( - (Success, makeMapStatus("hostA", 2)), - (Success, makeMapStatus("hostB", 2)))) - assert(mapOutputTracker.findMissingPartitions(shuffleId) === Some(Seq.empty)) - - runEvent(makeCompletionEvent( - taskSets(1).tasks(0), - FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, "ignored"), - null)) - - // Shuffle blocks of "hostA" is lost, so first task of the `shuffleMapRdd` needs to retry. - // The result stage is still waiting for its 2 tasks to complete. - // Because of shuffleMapRdd is indeterminate, this job will be abort. - assert(failure != null && failure.getMessage - .contains("Spark cannot rollback the ShuffleMapStage 0")) - } - test("Completions in zombie tasksets update status of non-zombie taskset") { val parts = 4 val shuffleMapRdd = new MyRDD(sc, parts, Nil) @@ -3036,8 +3137,9 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi } object DAGSchedulerSuite { - def makeMapStatus(host: String, reduces: Int, sizes: Byte = 2): MapStatus = - MapStatus(makeBlockManagerId(host), Array.fill[Long](reduces)(sizes)) + def makeMapStatus(host: String, reduces: Int, + sizes: Byte = 2, taskAttemptId: Long = -1): MapStatus = + MapStatus(makeBlockManagerId(host), Array.fill[Long](reduces)(sizes), taskAttemptId) def makeBlockManagerId(host: String): BlockManagerId = BlockManagerId("exec-" + host, host, 12345) diff --git a/core/src/test/scala/org/apache/spark/scheduler/MapStatusSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/MapStatusSuite.scala index c1e7fb9a1db16..886535ba4aa97 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/MapStatusSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/MapStatusSuite.scala @@ -61,7 +61,7 @@ class MapStatusSuite extends SparkFunSuite { stddev <- Seq(0.0, 0.01, 0.5, 1.0) ) { val sizes = Array.fill[Long](numSizes)(abs(round(Random.nextGaussian() * stddev)) + mean) - val status = MapStatus(BlockManagerId("a", "b", 10), sizes) + val status = MapStatus(BlockManagerId("a", "b", 10), sizes, -1) val status1 = compressAndDecompressMapStatus(status) for (i <- 0 until numSizes) { if (sizes(i) != 0) { @@ -75,7 +75,7 @@ class MapStatusSuite extends SparkFunSuite { test("large tasks should use " + classOf[HighlyCompressedMapStatus].getName) { val sizes = Array.fill[Long](2001)(150L) - val status = MapStatus(null, sizes) + val status = MapStatus(null, sizes, -1) assert(status.isInstanceOf[HighlyCompressedMapStatus]) assert(status.getSizeForBlock(10) === 150L) assert(status.getSizeForBlock(50) === 150L) @@ -87,10 +87,12 @@ class MapStatusSuite extends SparkFunSuite { val sizes = Array.tabulate[Long](3000) { i => i.toLong } val avg = sizes.sum / sizes.count(_ != 0) val loc = BlockManagerId("a", "b", 10) - val status = MapStatus(loc, sizes) + val mapTaskAttemptId = 5 + val status = MapStatus(loc, sizes, mapTaskAttemptId) val status1 = compressAndDecompressMapStatus(status) assert(status1.isInstanceOf[HighlyCompressedMapStatus]) assert(status1.location == loc) + assert(status1.mapTaskAttemptId == mapTaskAttemptId) for (i <- 0 until 3000) { val estimate = status1.getSizeForBlock(i) if (sizes(i) > 0) { @@ -109,7 +111,7 @@ class MapStatusSuite extends SparkFunSuite { val smallBlockSizes = sizes.filter(n => n > 0 && n < threshold) val avg = smallBlockSizes.sum / smallBlockSizes.length val loc = BlockManagerId("a", "b", 10) - val status = MapStatus(loc, sizes) + val status = MapStatus(loc, sizes, 5) val status1 = compressAndDecompressMapStatus(status) assert(status1.isInstanceOf[HighlyCompressedMapStatus]) assert(status1.location == loc) @@ -165,7 +167,7 @@ class MapStatusSuite extends SparkFunSuite { SparkEnv.set(env) // Value of element in sizes is equal to the corresponding index. val sizes = (0L to 2000L).toArray - val status1 = MapStatus(BlockManagerId("exec-0", "host-0", 100), sizes) + val status1 = MapStatus(BlockManagerId("exec-0", "host-0", 100), sizes, 5) val arrayStream = new ByteArrayOutputStream(102400) val objectOutputStream = new ObjectOutputStream(arrayStream) assert(status1.isInstanceOf[HighlyCompressedMapStatus]) diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala index 2442670b6d3f0..ff435a807801e 100644 --- a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala @@ -350,8 +350,11 @@ class KryoSerializerSuite extends SparkFunSuite with SharedSparkContext { val ser = new KryoSerializer(conf).newInstance() val denseBlockSizes = new Array[Long](5000) val sparseBlockSizes = Array[Long](0L, 1L, 0L, 2L) + var mapTaskAttemptId = 0 Seq(denseBlockSizes, sparseBlockSizes).foreach { blockSizes => - ser.serialize(HighlyCompressedMapStatus(BlockManagerId("exec-1", "host", 1234), blockSizes)) + mapTaskAttemptId += 1 + ser.serialize(HighlyCompressedMapStatus( + BlockManagerId("exec-1", "host", 1234), blockSizes, mapTaskAttemptId)) } } diff --git a/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderSuite.scala index 6d2ef17a7a790..bcd1dd105db9b 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderSuite.scala @@ -118,7 +118,7 @@ class BlockStoreShuffleReaderSuite extends SparkFunSuite with LocalSparkContext when(dependency.serializer).thenReturn(serializer) when(dependency.aggregator).thenReturn(None) when(dependency.keyOrdering).thenReturn(None) - new BaseShuffleHandle(shuffleId, numMaps, dependency) + new BaseShuffleHandle(shuffleId, dependency) } val serializerManager = new SerializerManager( diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala index b9f81fa0d0a06..808c0360ee99d 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala @@ -25,7 +25,7 @@ import scala.collection.mutable.ArrayBuffer import org.mockito.{Mock, MockitoAnnotations} import org.mockito.Answers.RETURNS_SMART_NULLS -import org.mockito.ArgumentMatchers.{any, anyInt} +import org.mockito.ArgumentMatchers.{any, anyInt, anyLong} import org.mockito.Mockito._ import org.scalatest.BeforeAndAfterEach @@ -65,7 +65,6 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte taskMetrics = new TaskMetrics shuffleHandle = new BypassMergeSortShuffleHandle[Int, Int]( shuffleId = 0, - numMaps = 2, dependency = dependency ) val memoryManager = new TestMemoryManager(conf) @@ -78,7 +77,7 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte when(taskContext.taskMemoryManager()).thenReturn(taskMemoryManager) when(blockResolver.writeIndexFileAndCommit( - anyInt, anyInt, any(classOf[Array[Long]]), any(classOf[File]))) + anyInt, anyLong, any(classOf[Array[Long]]), any(classOf[File]))) .thenAnswer { invocationOnMock => val tmp = invocationOnMock.getArguments()(3).asInstanceOf[File] if (tmp != null) { @@ -139,7 +138,6 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte val writer = new BypassMergeSortShuffleWriter[Int, Int]( blockManager, shuffleHandle, - 0, // MapId 0L, // MapTaskAttemptId conf, taskContext.taskMetrics().shuffleWriteMetrics, @@ -166,7 +164,6 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte val writer = new BypassMergeSortShuffleWriter[Int, Int]( blockManager, shuffleHandle, - 0, // MapId 0L, transferConf, taskContext.taskMetrics().shuffleWriteMetrics, @@ -202,7 +199,6 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte val writer = new BypassMergeSortShuffleWriter[Int, Int]( blockManager, shuffleHandle, - 0, // MapId 0L, conf, taskContext.taskMetrics().shuffleWriteMetrics, @@ -224,7 +220,6 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte val writer = new BypassMergeSortShuffleWriter[Int, Int]( blockManager, shuffleHandle, - 0, // MapId 0L, conf, taskContext.taskMetrics().shuffleWriteMetrics, diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleWriterSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleWriterSuite.scala index 0dd6040808f9e..e452b1a2a8f43 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleWriterSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleWriterSuite.scala @@ -57,7 +57,7 @@ class SortShuffleWriterSuite extends SparkFunSuite with SharedSparkContext with when(dependency.serializer).thenReturn(serializer) when(dependency.aggregator).thenReturn(None) when(dependency.keyOrdering).thenReturn(None) - new BaseShuffleHandle(shuffleId, numMaps = numMaps, dependency) + new BaseShuffleHandle(shuffleId, dependency) } shuffleExecutorComponents = new LocalDiskShuffleExecutorComponents( conf, blockManager, shuffleBlockResolver) @@ -72,11 +72,10 @@ class SortShuffleWriterSuite extends SparkFunSuite with SharedSparkContext with } test("write empty iterator") { - val context = MemoryTestingUtils.fakeTaskContext(sc.env) + val context = MemoryTestingUtils.fakeTaskContext(sc.env, taskAttemptId = 1) val writer = new SortShuffleWriter[Int, Int, Int]( shuffleBlockResolver, shuffleHandle, - mapId = 1, context, shuffleExecutorComponents) writer.write(Iterator.empty) @@ -89,12 +88,11 @@ class SortShuffleWriterSuite extends SparkFunSuite with SharedSparkContext with } test("write with some records") { - val context = MemoryTestingUtils.fakeTaskContext(sc.env) + val context = MemoryTestingUtils.fakeTaskContext(sc.env, taskAttemptId = 2) val records = List[(Int, Int)]((1, 2), (2, 3), (4, 4), (6, 5)) val writer = new SortShuffleWriter[Int, Int, Int]( shuffleBlockResolver, shuffleHandle, - mapId = 2, context, shuffleExecutorComponents) writer.write(records.toIterator) diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriterSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriterSuite.scala index 5156cc2cc47a6..f92455912f510 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriterSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriterSuite.scala @@ -23,7 +23,7 @@ import java.nio.file.Files import java.util.Arrays import org.mockito.Answers.RETURNS_SMART_NULLS -import org.mockito.ArgumentMatchers.{any, anyInt} +import org.mockito.ArgumentMatchers.{any, anyInt, anyLong} import org.mockito.Mock import org.mockito.Mockito.when import org.mockito.MockitoAnnotations @@ -73,9 +73,9 @@ class LocalDiskShuffleMapOutputWriterSuite extends SparkFunSuite with BeforeAndA conf = new SparkConf() .set("spark.app.id", "example.spark.app") .set("spark.shuffle.unsafe.file.output.buffer", "16k") - when(blockResolver.getDataFile(anyInt, anyInt)).thenReturn(mergedOutputFile) + when(blockResolver.getDataFile(anyInt, anyLong)).thenReturn(mergedOutputFile) when(blockResolver.writeIndexFileAndCommit( - anyInt, anyInt, any(classOf[Array[Long]]), any(classOf[File]))) + anyInt, anyLong, any(classOf[Array[Long]]), any(classOf[File]))) .thenAnswer { invocationOnMock => partitionSizesInMergedFile = invocationOnMock.getArguments()(2).asInstanceOf[Array[Long]] val tmp: File = invocationOnMock.getArguments()(3).asInstanceOf[File] diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 097f1d2c2a6e1..09499e04ac7f7 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -417,7 +417,25 @@ object MimaExcludes { // [SPARK-25382][SQL][PYSPARK] Remove ImageSchema.readImages in 3.0 ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.image.ImageSchema.readImages"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.image.ImageSchema.readImages") + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ml.image.ImageSchema.readImages"), + + // [SPARK-25341][CORE] Support rolling back a shuffle map stage and re-generate the shuffle files + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.shuffle.sort.UnsafeShuffleWriter.this"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.storage.ShuffleIndexBlockId.mapId"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.storage.ShuffleIndexBlockId.copy$default$2"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.storage.ShuffleIndexBlockId.copy"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.storage.ShuffleIndexBlockId.this"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.storage.ShuffleDataBlockId.mapId"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.storage.ShuffleDataBlockId.copy$default$2"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.storage.ShuffleDataBlockId.copy"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.storage.ShuffleDataBlockId.this"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.storage.ShuffleBlockId.mapId"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.storage.ShuffleBlockId.copy$default$2"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.storage.ShuffleBlockId.copy"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.storage.ShuffleBlockId.this"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.storage.ShuffleIndexBlockId.apply"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.storage.ShuffleDataBlockId.apply"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.storage.ShuffleBlockId.apply") ) // Exclude rules for 2.4.x From 578c2338c47819f7e247052c0aa7af6b389b2933 Mon Sep 17 00:00:00 2001 From: Yuanjian Li Date: Thu, 29 Aug 2019 22:09:58 +0800 Subject: [PATCH 02/15] Still need mapId for the fetch fail scenario --- .../org/apache/spark/MapOutputTracker.scala | 24 +++--- .../storage/ShuffleBlockFetcherIterator.scala | 85 +++++++++++-------- .../apache/spark/MapOutputTrackerSuite.scala | 16 ++-- .../BlockStoreShuffleReaderSuite.scala | 2 +- .../ShuffleBlockFetcherIteratorSuite.scala | 54 ++++++------ 5 files changed, 99 insertions(+), 82 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 4be82d31af764..6e805561cc532 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -282,7 +282,7 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging // For testing def getMapSizesByExecutorId(shuffleId: Int, reduceId: Int) - : Iterator[(BlockManagerId, Seq[(BlockId, Long)])] = { + : Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])] = { getMapSizesByExecutorId(shuffleId, reduceId, reduceId + 1) } @@ -292,11 +292,11 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging * endPartition is excluded from the range). * * @return A sequence of 2-item tuples, where the first item in the tuple is a BlockManagerId, - * and the second item is a sequence of (shuffle block id, shuffle block size) tuples - * describing the shuffle blocks that are stored at that block manager. + * and the second item is a sequence of (shuffle block id, shuffle block size, map id) + * tuples describing the shuffle blocks that are stored at that block manager. */ def getMapSizesByExecutorId(shuffleId: Int, startPartition: Int, endPartition: Int) - : Iterator[(BlockManagerId, Seq[(BlockId, Long)])] + : Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])] /** * Deletes map output status information for the specified shuffle stage. @@ -646,7 +646,7 @@ private[spark] class MapOutputTrackerMaster( // Get blocks sizes by executor Id. Note that zero-sized blocks are excluded in the result. // This method is only called in local-mode. def getMapSizesByExecutorId(shuffleId: Int, startPartition: Int, endPartition: Int) - : Iterator[(BlockManagerId, Seq[(BlockId, Long)])] = { + : Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])] = { logDebug(s"Fetching outputs for shuffle $shuffleId, partitions $startPartition-$endPartition") shuffleStatuses.get(shuffleId) match { case Some (shuffleStatus) => @@ -686,7 +686,7 @@ private[spark] class MapOutputTrackerWorker(conf: SparkConf) extends MapOutputTr // Get blocks sizes by executor Id. Note that zero-sized blocks are excluded in the result. override def getMapSizesByExecutorId(shuffleId: Int, startPartition: Int, endPartition: Int) - : Iterator[(BlockManagerId, Seq[(BlockId, Long)])] = { + : Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])] = { logDebug(s"Fetching outputs for shuffle $shuffleId, partitions $startPartition-$endPartition") val statuses = getStatuses(shuffleId) try { @@ -834,17 +834,17 @@ private[spark] object MapOutputTracker extends Logging { * @param endPartition End of map output partition ID range (excluded from range) * @param statuses List of map statuses, indexed by map ID. * @return A sequence of 2-item tuples, where the first item in the tuple is a BlockManagerId, - * and the second item is a sequence of (shuffle block ID, shuffle block size) tuples - * describing the shuffle blocks that are stored at that block manager. + * and the second item is a sequence of (shuffle block id, shuffle block size, map id) + * tuples describing the shuffle blocks that are stored at that block manager. */ def convertMapStatuses( shuffleId: Int, startPartition: Int, endPartition: Int, - statuses: Array[MapStatus]): Iterator[(BlockManagerId, Seq[(BlockId, Long)])] = { + statuses: Array[MapStatus]): Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])] = { assert (statuses != null) - val splitsByAddress = new HashMap[BlockManagerId, ListBuffer[(BlockId, Long)]] - statuses.foreach { status => + val splitsByAddress = new HashMap[BlockManagerId, ListBuffer[(BlockId, Long, Int)]] + for ((status, mapId) <- statuses.iterator.zipWithIndex) { if (status == null) { val errorMessage = s"Missing an output location for shuffle $shuffleId" logError(errorMessage) @@ -854,7 +854,7 @@ private[spark] object MapOutputTracker extends Logging { val size = status.getSizeForBlock(part) if (size != 0) { splitsByAddress.getOrElseUpdate(status.location, ListBuffer()) += - ((ShuffleBlockId(shuffleId, status.mapTaskAttemptId, part), size)) + ((ShuffleBlockId(shuffleId, status.mapTaskAttemptId, part), size, mapId)) } } } diff --git a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala index 8a6c199423506..e45962f7ffebf 100644 --- a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala @@ -49,9 +49,10 @@ import org.apache.spark.util.{CompletionIterator, TaskCompletionListener, Utils} * @param shuffleClient [[BlockStoreClient]] for fetching remote blocks * @param blockManager [[BlockManager]] for reading local blocks * @param blocksByAddress list of blocks to fetch grouped by the [[BlockManagerId]]. - * For each block we also require the size (in bytes as a long field) in - * order to throttle the memory usage. Note that zero-sized blocks are - * already excluded, which happened in + * For each block we also require two info: 1. the size (in bytes as a long + * field) in order to throttle the memory usage; 2. the mapId for this + * block, which indicate the index in the map stage of the block. + * Note that zero-sized blocks are already excluded, which happened in * [[org.apache.spark.MapOutputTracker.convertMapStatuses]]. * @param streamWrapper A function to wrap the returned input stream. * @param maxBytesInFlight max size (in bytes) of remote blocks to fetch at any given point. @@ -67,7 +68,7 @@ final class ShuffleBlockFetcherIterator( context: TaskContext, shuffleClient: BlockStoreClient, blockManager: BlockManager, - blocksByAddress: Iterator[(BlockManagerId, Seq[(BlockId, Long)])], + blocksByAddress: Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])], streamWrapper: (BlockId, InputStream) => InputStream, maxBytesInFlight: Long, maxReqsInFlight: Int, @@ -97,7 +98,7 @@ final class ShuffleBlockFetcherIterator( private[this] val startTimeNs = System.nanoTime() /** Local blocks to fetch, excluding zero-sized blocks. */ - private[this] val localBlocks = scala.collection.mutable.LinkedHashSet[BlockId]() + private[this] val localBlocks = scala.collection.mutable.LinkedHashSet[(BlockId, Int)]() /** Remote blocks to fetch, excluding zero-sized blocks. */ private[this] val remoteBlocks = new HashSet[BlockId]() @@ -199,7 +200,7 @@ final class ShuffleBlockFetcherIterator( while (iter.hasNext) { val result = iter.next() result match { - case SuccessFetchResult(_, address, _, buf, _) => + case SuccessFetchResult(_, _, address, _, buf, _) => if (address != blockManager.blockManagerId) { shuffleMetrics.incRemoteBytesRead(buf.size) if (buf.isInstanceOf[FileSegmentManagedBuffer]) { @@ -224,9 +225,11 @@ final class ShuffleBlockFetcherIterator( bytesInFlight += req.size reqsInFlight += 1 - // so we can look up the size of each blockID - val sizeMap = req.blocks.map { case (blockId, size) => (blockId.toString, size) }.toMap - val remainingBlocks = new HashSet[String]() ++= sizeMap.keys + // so we can look up the block info of each blockID + val infoMap = req.blocks.map { + case (blockId, size, mapId) => (blockId.toString, (size, mapId)) + }.toMap + val remainingBlocks = new HashSet[String]() ++= infoMap.keys val blockIds = req.blocks.map(_._1.toString) val address = req.address @@ -240,8 +243,8 @@ final class ShuffleBlockFetcherIterator( // This needs to be released after use. buf.retain() remainingBlocks -= blockId - results.put(new SuccessFetchResult(BlockId(blockId), address, sizeMap(blockId), buf, - remainingBlocks.isEmpty)) + results.put(new SuccessFetchResult(BlockId(blockId), infoMap(blockId)._2, + address, infoMap(blockId)._1, buf, remainingBlocks.isEmpty)) logDebug("remainingBlocks: " + remainingBlocks) } } @@ -250,7 +253,7 @@ final class ShuffleBlockFetcherIterator( override def onBlockFetchFailure(blockId: String, e: Throwable): Unit = { logError(s"Failed to get block(s) from ${req.address.host}:${req.address.port}", e) - results.put(new FailureFetchResult(BlockId(blockId), address, e)) + results.put(new FailureFetchResult(BlockId(blockId), infoMap(blockId)._2, address, e)) } } @@ -283,28 +286,28 @@ final class ShuffleBlockFetcherIterator( for ((address, blockInfos) <- blocksByAddress) { if (address.executorId == blockManager.blockManagerId.executorId) { blockInfos.find(_._2 <= 0) match { - case Some((blockId, size)) if size < 0 => + case Some((blockId, size, _)) if size < 0 => throw new BlockException(blockId, "Negative block size " + size) - case Some((blockId, size)) if size == 0 => + case Some((blockId, size, _)) if size == 0 => throw new BlockException(blockId, "Zero-sized blocks should be excluded.") case None => // do nothing. } - localBlocks ++= blockInfos.map(_._1) + localBlocks ++= blockInfos.map(info => (info._1, info._3)) localBlockBytes += blockInfos.map(_._2).sum numBlocksToFetch += localBlocks.size } else { val iterator = blockInfos.iterator var curRequestSize = 0L - var curBlocks = new ArrayBuffer[(BlockId, Long)] + var curBlocks = new ArrayBuffer[(BlockId, Long, Int)] while (iterator.hasNext) { - val (blockId, size) = iterator.next() + val (blockId, size, mapId) = iterator.next() remoteBlockBytes += size if (size < 0) { throw new BlockException(blockId, "Negative block size " + size) } else if (size == 0) { throw new BlockException(blockId, "Zero-sized blocks should be excluded.") } else { - curBlocks += ((blockId, size)) + curBlocks += ((blockId, size, mapId)) remoteBlocks += blockId numBlocksToFetch += 1 curRequestSize += size @@ -315,7 +318,7 @@ final class ShuffleBlockFetcherIterator( remoteRequests += new FetchRequest(address, curBlocks) logDebug(s"Creating fetch request of $curRequestSize at $address " + s"with ${curBlocks.size} blocks") - curBlocks = new ArrayBuffer[(BlockId, Long)] + curBlocks = new ArrayBuffer[(BlockId, Long, Int)] curRequestSize = 0 } } @@ -341,13 +344,13 @@ final class ShuffleBlockFetcherIterator( logDebug(s"Start fetching local blocks: ${localBlocks.mkString(", ")}") val iter = localBlocks.iterator while (iter.hasNext) { - val blockId = iter.next() + val (blockId, mapId) = iter.next() try { val buf = blockManager.getBlockData(blockId) shuffleMetrics.incLocalBlocksFetched(1) shuffleMetrics.incLocalBytesRead(buf.size) buf.retain() - results.put(new SuccessFetchResult(blockId, blockManager.blockManagerId, + results.put(new SuccessFetchResult(blockId, mapId, blockManager.blockManagerId, buf.size(), buf, false)) } catch { // If we see an exception, stop immediately. @@ -360,7 +363,7 @@ final class ShuffleBlockFetcherIterator( logError("Error occurred while fetching local blocks, " + ce.getMessage) case ex: Exception => logError("Error occurred while fetching local blocks", ex) } - results.put(new FailureFetchResult(blockId, blockManager.blockManagerId, e)) + results.put(new FailureFetchResult(blockId, mapId, blockManager.blockManagerId, e)) return } } @@ -420,7 +423,7 @@ final class ShuffleBlockFetcherIterator( shuffleMetrics.incFetchWaitTime(fetchWaitTime) result match { - case r @ SuccessFetchResult(blockId, address, size, buf, isNetworkReqDone) => + case r @ SuccessFetchResult(blockId, mapId, address, size, buf, isNetworkReqDone) => if (address != blockManager.blockManagerId) { numBlocksInFlightPerAddress(address) = numBlocksInFlightPerAddress(address) - 1 shuffleMetrics.incRemoteBytesRead(buf.size) @@ -429,7 +432,7 @@ final class ShuffleBlockFetcherIterator( } shuffleMetrics.incRemoteBlocksFetched(1) } - if (!localBlocks.contains(blockId)) { + if (!localBlocks.contains((blockId, mapId))) { bytesInFlight -= size } if (isNetworkReqDone) { @@ -453,7 +456,7 @@ final class ShuffleBlockFetcherIterator( // since the last call. val msg = s"Received a zero-size buffer for block $blockId from $address " + s"(expectedApproxSize = $size, isNetworkReqDone=$isNetworkReqDone)" - throwFetchFailedException(blockId, address, new IOException(msg)) + throwFetchFailedException(blockId, mapId, address, new IOException(msg)) } val in = try { @@ -469,7 +472,7 @@ final class ShuffleBlockFetcherIterator( case e: IOException => logError("Failed to create input stream from local block", e) } buf.release() - throwFetchFailedException(blockId, address, e) + throwFetchFailedException(blockId, mapId, address, e) } try { input = streamWrapper(blockId, in) @@ -487,11 +490,11 @@ final class ShuffleBlockFetcherIterator( buf.release() if (buf.isInstanceOf[FileSegmentManagedBuffer] || corruptedBlocks.contains(blockId)) { - throwFetchFailedException(blockId, address, e) + throwFetchFailedException(blockId, mapId, address, e) } else { logWarning(s"got an corrupted block $blockId from $address, fetch again", e) corruptedBlocks += blockId - fetchRequests += FetchRequest(address, Array((blockId, size))) + fetchRequests += FetchRequest(address, Array((blockId, size, mapId))) result = null } } finally { @@ -503,8 +506,8 @@ final class ShuffleBlockFetcherIterator( } } - case FailureFetchResult(blockId, address, e) => - throwFetchFailedException(blockId, address, e) + case FailureFetchResult(blockId, mapId, address, e) => + throwFetchFailedException(blockId, mapId, address, e) } // Send fetch requests up to maxBytesInFlight @@ -517,6 +520,7 @@ final class ShuffleBlockFetcherIterator( input, this, currentResult.blockId, + currentResult.mapId, currentResult.address, detectCorrupt && streamCompressedOrEncrypted)) } @@ -583,10 +587,11 @@ final class ShuffleBlockFetcherIterator( private[storage] def throwFetchFailedException( blockId: BlockId, + mapId: Int, address: BlockManagerId, e: Throwable) = { blockId match { - case ShuffleBlockId(shufId, mapId, reduceId) => + case ShuffleBlockId(shufId, _, reduceId) => throw new FetchFailedException(address, shufId.toInt, mapId.toInt, reduceId, e) case _ => throw new SparkException( @@ -604,6 +609,7 @@ private class BufferReleasingInputStream( private[storage] val delegate: InputStream, private val iterator: ShuffleBlockFetcherIterator, private val blockId: BlockId, + private val mapId: Int, private val address: BlockManagerId, private val detectCorruption: Boolean) extends InputStream { @@ -615,7 +621,7 @@ private class BufferReleasingInputStream( } catch { case e: IOException if detectCorruption => IOUtils.closeQuietly(this) - iterator.throwFetchFailedException(blockId, address, e) + iterator.throwFetchFailedException(blockId, mapId, address, e) } } @@ -637,7 +643,7 @@ private class BufferReleasingInputStream( } catch { case e: IOException if detectCorruption => IOUtils.closeQuietly(this) - iterator.throwFetchFailedException(blockId, address, e) + iterator.throwFetchFailedException(blockId, mapId, address, e) } } @@ -649,7 +655,7 @@ private class BufferReleasingInputStream( } catch { case e: IOException if detectCorruption => IOUtils.closeQuietly(this) - iterator.throwFetchFailedException(blockId, address, e) + iterator.throwFetchFailedException(blockId, mapId, address, e) } } @@ -659,7 +665,7 @@ private class BufferReleasingInputStream( } catch { case e: IOException if detectCorruption => IOUtils.closeQuietly(this) - iterator.throwFetchFailedException(blockId, address, e) + iterator.throwFetchFailedException(blockId, mapId, address, e) } } @@ -694,9 +700,10 @@ object ShuffleBlockFetcherIterator { * A request to fetch blocks from a remote BlockManager. * @param address remote BlockManager to fetch from. * @param blocks Sequence of tuple, where the first element is the block id, - * and the second element is the estimated size, used to calculate bytesInFlight. + * and the second element is the estimated size, used to calculate bytesInFlight, + * the third element is the mapId. */ - case class FetchRequest(address: BlockManagerId, blocks: Seq[(BlockId, Long)]) { + case class FetchRequest(address: BlockManagerId, blocks: Seq[(BlockId, Long, Int)]) { val size = blocks.map(_._2).sum } @@ -711,6 +718,7 @@ object ShuffleBlockFetcherIterator { /** * Result of a fetch from a remote block successfully. * @param blockId block id + * @param mapId mapId for this block * @param address BlockManager that the block was fetched from. * @param size estimated size of the block. Note that this is NOT the exact bytes. * Size of remote block is used to calculate bytesInFlight. @@ -719,6 +727,7 @@ object ShuffleBlockFetcherIterator { */ private[storage] case class SuccessFetchResult( blockId: BlockId, + mapId: Int, address: BlockManagerId, size: Long, buf: ManagedBuffer, @@ -730,11 +739,13 @@ object ShuffleBlockFetcherIterator { /** * Result of a fetch from a remote block unsuccessfully. * @param blockId block id + * @param mapId mapId for this block * @param address BlockManager that the block was attempted to be fetched from * @param e the failure exception */ private[storage] case class FailureFetchResult( blockId: BlockId, + mapId: Int, address: BlockManagerId, e: Throwable) extends FetchResult diff --git a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala index 06602563693a1..d5ee19bde8edf 100644 --- a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala @@ -69,9 +69,10 @@ class MapOutputTrackerSuite extends SparkFunSuite { Array(10000L, 1000L), 6)) val statuses = tracker.getMapSizesByExecutorId(10, 0) assert(statuses.toSet === - Seq((BlockManagerId("a", "hostA", 1000), ArrayBuffer((ShuffleBlockId(10, 5, 0), size1000))), - (BlockManagerId("b", "hostB", 1000), ArrayBuffer((ShuffleBlockId(10, 6, 0), size10000)))) - .toSet) + Seq((BlockManagerId("a", "hostA", 1000), + ArrayBuffer((ShuffleBlockId(10, 5, 0), size1000, 0))), + (BlockManagerId("b", "hostB", 1000), + ArrayBuffer((ShuffleBlockId(10, 6, 0), size10000, 1)))).toSet) assert(0 == tracker.getNumCachedSerializedBroadcast) tracker.stop() rpcEnv.shutdown() @@ -150,7 +151,8 @@ class MapOutputTrackerSuite extends SparkFunSuite { BlockManagerId("a", "hostA", 1000), Array(1000L), 5)) slaveTracker.updateEpoch(masterTracker.getEpoch) assert(slaveTracker.getMapSizesByExecutorId(10, 0).toSeq === - Seq((BlockManagerId("a", "hostA", 1000), ArrayBuffer((ShuffleBlockId(10, 5, 0), size1000))))) + Seq((BlockManagerId("a", "hostA", 1000), + ArrayBuffer((ShuffleBlockId(10, 5, 0), size1000, 0))))) assert(0 == masterTracker.getNumCachedSerializedBroadcast) val masterTrackerEpochBeforeLossOfMapOutput = masterTracker.getEpoch @@ -318,9 +320,11 @@ class MapOutputTrackerSuite extends SparkFunSuite { assert(tracker.getMapSizesByExecutorId(10, 0, 4).toSeq === Seq( (BlockManagerId("a", "hostA", 1000), - Seq((ShuffleBlockId(10, 5, 1), size1000), (ShuffleBlockId(10, 5, 3), size10000))), + Seq((ShuffleBlockId(10, 5, 1), size1000, 0), + (ShuffleBlockId(10, 5, 3), size10000, 0))), (BlockManagerId("b", "hostB", 1000), - Seq((ShuffleBlockId(10, 6, 0), size10000), (ShuffleBlockId(10, 6, 2), size1000))) + Seq((ShuffleBlockId(10, 6, 0), size10000, 1), + (ShuffleBlockId(10, 6, 2), size1000, 1))) ) ) diff --git a/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderSuite.scala index bcd1dd105db9b..1a576d82a0a08 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderSuite.scala @@ -107,7 +107,7 @@ class BlockStoreShuffleReaderSuite extends SparkFunSuite with LocalSparkContext // for the code to read data over the network. val shuffleBlockIdsAndSizes = (0 until numMaps).map { mapId => val shuffleBlockId = ShuffleBlockId(shuffleId, mapId, reduceId) - (shuffleBlockId, byteOutputStream.size().toLong) + (shuffleBlockId, byteOutputStream.size().toLong, mapId) } Seq((localBlockManagerId, shuffleBlockIdsAndSizes)).toIterator } diff --git a/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala b/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala index ed402440e74f1..05c21492b93e9 100644 --- a/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala @@ -98,9 +98,9 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT val transfer = createMockTransfer(remoteBlocks) - val blocksByAddress = Seq[(BlockManagerId, Seq[(BlockId, Long)])]( - (localBmId, localBlocks.keys.map(blockId => (blockId, 1.asInstanceOf[Long])).toSeq), - (remoteBmId, remoteBlocks.keys.map(blockId => (blockId, 1.asInstanceOf[Long])).toSeq) + val blocksByAddress = Seq[(BlockManagerId, Seq[(BlockId, Long, Int)])]( + (localBmId, localBlocks.keys.map(blockId => (blockId, 1.asInstanceOf[Long], 0)).toSeq), + (remoteBmId, remoteBlocks.keys.map(blockId => (blockId, 1.asInstanceOf[Long], 1)).toSeq) ).toIterator val taskContext = TaskContext.empty() @@ -179,8 +179,8 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT } }) - val blocksByAddress = Seq[(BlockManagerId, Seq[(BlockId, Long)])]( - (remoteBmId, blocks.keys.map(blockId => (blockId, 1.asInstanceOf[Long])).toSeq)).toIterator + val blocksByAddress = Seq[(BlockManagerId, Seq[(BlockId, Long, Int)])]( + (remoteBmId, blocks.keys.map(blockId => (blockId, 1.asInstanceOf[Long], 0)).toSeq)).toIterator val taskContext = TaskContext.empty() val iterator = new ShuffleBlockFetcherIterator( @@ -247,8 +247,9 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT } }) - val blocksByAddress = Seq[(BlockManagerId, Seq[(BlockId, Long)])]( - (remoteBmId, blocks.keys.map(blockId => (blockId, 1.asInstanceOf[Long])).toSeq)).toIterator + val blocksByAddress = Seq[(BlockManagerId, Seq[(BlockId, Long, Int)])]( + (remoteBmId, blocks.keys.map(blockId => (blockId, 1.asInstanceOf[Long], 0)).toSeq)) + .toIterator val taskContext = TaskContext.empty() val iterator = new ShuffleBlockFetcherIterator( @@ -336,8 +337,8 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT } }) - val blocksByAddress = Seq[(BlockManagerId, Seq[(BlockId, Long)])]( - (remoteBmId, blocks.keys.map(blockId => (blockId, 1.asInstanceOf[Long])).toSeq)).toIterator + val blocksByAddress = Seq[(BlockManagerId, Seq[(BlockId, Long, Int)])]( + (remoteBmId, blocks.keys.map(blockId => (blockId, 1.asInstanceOf[Long], 0)).toSeq)).toIterator val taskContext = TaskContext.empty() val iterator = new ShuffleBlockFetcherIterator( @@ -389,8 +390,8 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT val corruptBuffer1 = mockCorruptBuffer(streamLength, 0) val blockManagerId1 = BlockManagerId("remote-client-1", "remote-client-1", 1) val shuffleBlockId1 = ShuffleBlockId(0, 1, 0) - val blockLengths1 = Seq[Tuple2[BlockId, Long]]( - shuffleBlockId1 -> corruptBuffer1.size() + val blockLengths1 = Seq[Tuple3[BlockId, Long, Int]]( + (shuffleBlockId1, corruptBuffer1.size(), 1) ) val streamNotCorruptTill = 8 * 1024 @@ -398,13 +399,13 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT val corruptBuffer2 = mockCorruptBuffer(streamLength, streamNotCorruptTill) val blockManagerId2 = BlockManagerId("remote-client-2", "remote-client-2", 2) val shuffleBlockId2 = ShuffleBlockId(0, 2, 0) - val blockLengths2 = Seq[Tuple2[BlockId, Long]]( - shuffleBlockId2 -> corruptBuffer2.size() + val blockLengths2 = Seq[Tuple3[BlockId, Long, Int]]( + (shuffleBlockId2, corruptBuffer2.size(), 2) ) val transfer = createMockTransfer( Map(shuffleBlockId1 -> corruptBuffer1, shuffleBlockId2 -> corruptBuffer2)) - val blocksByAddress = Seq[(BlockManagerId, Seq[(BlockId, Long)])]( + val blocksByAddress = Seq[(BlockManagerId, Seq[(BlockId, Long, Int)])]( (blockManagerId1, blockLengths1), (blockManagerId2, blockLengths2) ).toIterator @@ -465,11 +466,11 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT val localBmId = BlockManagerId("test-client", "test-client", 1) doReturn(localBmId).when(blockManager).blockManagerId doReturn(managedBuffer).when(blockManager).getBlockData(ShuffleBlockId(0, 0, 0)) - val localBlockLengths = Seq[Tuple2[BlockId, Long]]( - ShuffleBlockId(0, 0, 0) -> 10000 + val localBlockLengths = Seq[Tuple3[BlockId, Long, Int]]( + (ShuffleBlockId(0, 0, 0), 10000, 0) ) val transfer = createMockTransfer(Map(ShuffleBlockId(0, 0, 0) -> managedBuffer)) - val blocksByAddress = Seq[(BlockManagerId, Seq[(BlockId, Long)])]( + val blocksByAddress = Seq[(BlockManagerId, Seq[(BlockId, Long, Int)])]( (localBmId, localBlockLengths) ).toIterator @@ -531,8 +532,9 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT } }) - val blocksByAddress = Seq[(BlockManagerId, Seq[(BlockId, Long)])]( - (remoteBmId, blocks.keys.map(blockId => (blockId, 1.asInstanceOf[Long])).toSeq)).toIterator + val blocksByAddress = Seq[(BlockManagerId, Seq[(BlockId, Long, Int)])]( + (remoteBmId, blocks.keys.map(blockId => (blockId, 1.asInstanceOf[Long], 0)).toSeq)) + .toIterator val taskContext = TaskContext.empty() val iterator = new ShuffleBlockFetcherIterator( @@ -591,7 +593,7 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT }) def fetchShuffleBlock( - blocksByAddress: Iterator[(BlockManagerId, Seq[(BlockId, Long)])]): Unit = { + blocksByAddress: Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])]): Unit = { // Set `maxBytesInFlight` and `maxReqsInFlight` to `Int.MaxValue`, so that during the // construction of `ShuffleBlockFetcherIterator`, all requests to fetch remote shuffle blocks // are issued. The `maxReqSizeShuffleToMem` is hard-coded as 200 here. @@ -611,15 +613,15 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT taskContext.taskMetrics.createTempShuffleReadMetrics()) } - val blocksByAddress1 = Seq[(BlockManagerId, Seq[(BlockId, Long)])]( - (remoteBmId, remoteBlocks.keys.map(blockId => (blockId, 100L)).toSeq)).toIterator + val blocksByAddress1 = Seq[(BlockManagerId, Seq[(BlockId, Long, Int)])]( + (remoteBmId, remoteBlocks.keys.map(blockId => (blockId, 100L, 0)).toSeq)).toIterator fetchShuffleBlock(blocksByAddress1) // `maxReqSizeShuffleToMem` is 200, which is greater than the block size 100, so don't fetch // shuffle block to disk. assert(tempFileManager == null) - val blocksByAddress2 = Seq[(BlockManagerId, Seq[(BlockId, Long)])]( - (remoteBmId, remoteBlocks.keys.map(blockId => (blockId, 300L)).toSeq)).toIterator + val blocksByAddress2 = Seq[(BlockManagerId, Seq[(BlockId, Long, Int)])]( + (remoteBmId, remoteBlocks.keys.map(blockId => (blockId, 300L, 0)).toSeq)).toIterator fetchShuffleBlock(blocksByAddress2) // `maxReqSizeShuffleToMem` is 200, which is smaller than the block size 300, so fetch // shuffle block to disk. @@ -640,8 +642,8 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT val transfer = createMockTransfer(blocks.mapValues(_ => createMockManagedBuffer(0))) - val blocksByAddress = Seq[(BlockManagerId, Seq[(BlockId, Long)])]( - (remoteBmId, blocks.keys.map(blockId => (blockId, 1.asInstanceOf[Long])).toSeq)) + val blocksByAddress = Seq[(BlockManagerId, Seq[(BlockId, Long, Int)])]( + (remoteBmId, blocks.keys.map(blockId => (blockId, 1.asInstanceOf[Long], 0)).toSeq)) val taskContext = TaskContext.empty() val iterator = new ShuffleBlockFetcherIterator( From cb612e5d153323866506b363e4ea4f778d024429 Mon Sep 17 00:00:00 2001 From: Yuanjian Li Date: Thu, 29 Aug 2019 22:44:33 +0800 Subject: [PATCH 03/15] fix doc --- .../org/apache/spark/shuffle/api/ShuffleMapOutputWriter.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/java/org/apache/spark/shuffle/api/ShuffleMapOutputWriter.java b/core/src/main/java/org/apache/spark/shuffle/api/ShuffleMapOutputWriter.java index 7fac00b7fbc3f..21abe9a57cd25 100644 --- a/core/src/main/java/org/apache/spark/shuffle/api/ShuffleMapOutputWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/api/ShuffleMapOutputWriter.java @@ -39,7 +39,7 @@ public interface ShuffleMapOutputWriter { * for the same partition within any given map task. The partition identifier will be in the * range of precisely 0 (inclusive) to numPartitions (exclusive), where numPartitions was * provided upon the creation of this map output writer via - * {@link ShuffleExecutorComponents#createMapOutputWriter(int, int, long, int)}. + * {@link ShuffleExecutorComponents#createMapOutputWriter(int, long, int)}. *

* Calls to this method will be invoked with monotonically increasing reducePartitionIds; each * call to this method will be called with a reducePartitionId that is strictly greater than From f4471b24063cdcdaf12f72608a5542f0091fca53 Mon Sep 17 00:00:00 2001 From: Yuanjian Li Date: Fri, 30 Aug 2019 15:40:58 +0800 Subject: [PATCH 04/15] Comment address and bug fix --- .../network/shuffle/ExternalBlockHandler.java | 58 ++++++++++++++----- .../api/ShuffleExecutorComponents.java | 8 +-- .../sort/BypassMergeSortShuffleWriter.java | 12 ++-- .../shuffle/sort/UnsafeShuffleWriter.java | 7 +-- .../LocalDiskShuffleExecutorComponents.java | 4 +- .../io/LocalDiskShuffleMapOutputWriter.java | 11 ++-- .../org/apache/spark/MapOutputTracker.scala | 51 +++++++++++----- .../apache/spark/scheduler/MapStatus.scala | 33 ++++++----- .../shuffle/BlockStoreShuffleReader.scala | 3 +- .../shuffle/sort/SortShuffleWriter.scala | 8 +-- .../apache/spark/MapOutputTrackerSuite.scala | 2 +- .../spark/memory/MemoryTestingUtils.scala | 4 +- .../spark/scheduler/DAGSchedulerSuite.scala | 13 ++--- .../spark/scheduler/MapStatusSuite.scala | 2 +- .../BlockStoreShuffleReaderSuite.scala | 3 +- .../shuffle/sort/SortShuffleWriterSuite.scala | 4 +- 16 files changed, 136 insertions(+), 87 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 9ba78808415e8..19cfac4849b9d 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 @@ -106,7 +106,7 @@ protected void handleMessage( numBlockIds += ids.length; } streamId = streamManager.registerStream(client.getClientId(), - new ManagedBufferIterator(msg, numBlockIds), client.getChannel()); + new ShuffleManagedBufferIterator(msg), client.getChannel()); } else { // For the compatibility with the old version, still keep the support for OpenBlocks. OpenBlocks msg = (OpenBlocks) msgObj; @@ -299,21 +299,6 @@ private int[] shuffleMapIdAndReduceIds(String[] blockIds, int shuffleId) { return mapIdAndReduceIds; } - ManagedBufferIterator(FetchShuffleBlocks msg, int numBlockIds) { - final long[] mapIdAndReduceIds = new long[2 * numBlockIds]; - int idx = 0; - for (int i = 0; i < msg.mapIds.length; i++) { - for (int reduceId : msg.reduceIds[i]) { - mapIdAndReduceIds[idx++] = msg.mapIds[i]; - mapIdAndReduceIds[idx++] = reduceId; - } - } - assert(idx == 2 * numBlockIds); - size = mapIdAndReduceIds.length; - blockDataForIndexFn = index -> blockManager.getBlockData(msg.appId, msg.execId, - msg.shuffleId, mapIdAndReduceIds[index], (int) mapIdAndReduceIds[index + 1]); - } - @Override public boolean hasNext() { return index < size; @@ -328,6 +313,47 @@ public ManagedBuffer next() { } } + private class ShuffleManagedBufferIterator implements Iterator { + + private int mapIdx = 0; + private int reduceIdx = 0; + + private final String appId; + private final String execId; + private final int shuffleId; + private final long[] mapIds; + private final int[][] reduceIds; + + ShuffleManagedBufferIterator(FetchShuffleBlocks msg) { + appId = msg.appId; + execId = msg.execId; + shuffleId = msg.shuffleId; + mapIds = msg.mapIds; + reduceIds = msg.reduceIds; + } + + @Override + public boolean hasNext() { + // mapIds.length must equal to reduceIds.length, and the passed in FetchShuffleBlocks + // must have non-empty mapIds and reduceIds, see the checking logic in OneForOneBlockFetcher. + return mapIdx < mapIds.length && reduceIdx < reduceIds[mapIdx].length; + } + + @Override + public ManagedBuffer next() { + final ManagedBuffer block = blockManager.getBlockData( + appId, execId, shuffleId, mapIds[mapIdx], reduceIds[mapIdx][reduceIdx]); + reduceIdx += 1; + if (reduceIdx == reduceIds[mapIdx].length) { + // Reach the end of current mapId, move to next mapId and its reduceIds. + reduceIdx = 0; + mapIdx += 1; + } + metrics.blockTransferRateBytes.mark(block != null ? block.size() : 0); + return block; + } + } + @Override public void channelActive(TransportClient client) { metrics.activeConnections.inc(); diff --git a/core/src/main/java/org/apache/spark/shuffle/api/ShuffleExecutorComponents.java b/core/src/main/java/org/apache/spark/shuffle/api/ShuffleExecutorComponents.java index 7f2258daa11d7..adbb1b773bf82 100644 --- a/core/src/main/java/org/apache/spark/shuffle/api/ShuffleExecutorComponents.java +++ b/core/src/main/java/org/apache/spark/shuffle/api/ShuffleExecutorComponents.java @@ -41,15 +41,15 @@ public interface ShuffleExecutorComponents { * Called once per map task to create a writer that will be responsible for persisting all the * partitioned bytes written by that map task. * @param shuffleId Unique identifier for the shuffle the map task is a part of - * @param mapTaskAttemptId Identifier of the task attempt. Multiple attempts of the same map task - * with the same (shuffleId, mapId) pair can be distinguished by the - * different values of mapTaskAttemptId. + * @param mapId Identifier of the task attempt. Multiple attempts of the same map task with the + * same (shuffleId, mapIndex) pair can be distinguished by the different values of + * mapId. * @param numPartitions The number of partitions that will be written by the map task. Some of * these partitions may be empty. */ ShuffleMapOutputWriter createMapOutputWriter( int shuffleId, - long mapTaskAttemptId, + long mapId, int numPartitions) throws IOException; /** diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java index 03e90619516ca..dc157eaa3b253 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java @@ -85,7 +85,7 @@ final class BypassMergeSortShuffleWriter extends ShuffleWriter { private final Partitioner partitioner; private final ShuffleWriteMetricsReporter writeMetrics; private final int shuffleId; - private final long mapTaskAttemptId; + private final long mapId; private final Serializer serializer; private final ShuffleExecutorComponents shuffleExecutorComponents; @@ -105,7 +105,7 @@ final class BypassMergeSortShuffleWriter extends ShuffleWriter { BypassMergeSortShuffleWriter( BlockManager blockManager, BypassMergeSortShuffleHandle handle, - long mapTaskAttemptId, + long mapId, SparkConf conf, ShuffleWriteMetricsReporter writeMetrics, ShuffleExecutorComponents shuffleExecutorComponents) { @@ -114,7 +114,7 @@ final class BypassMergeSortShuffleWriter extends ShuffleWriter { this.transferToEnabled = conf.getBoolean("spark.file.transferTo", true); this.blockManager = blockManager; final ShuffleDependency dep = handle.dependency(); - this.mapTaskAttemptId = mapTaskAttemptId; + this.mapId = mapId; this.shuffleId = dep.shuffleId(); this.partitioner = dep.partitioner(); this.numPartitions = partitioner.numPartitions(); @@ -127,12 +127,12 @@ final class BypassMergeSortShuffleWriter extends ShuffleWriter { public void write(Iterator> records) throws IOException { assert (partitionWriters == null); ShuffleMapOutputWriter mapOutputWriter = shuffleExecutorComponents - .createMapOutputWriter(shuffleId, mapTaskAttemptId, numPartitions); + .createMapOutputWriter(shuffleId, mapId, numPartitions); try { if (!records.hasNext()) { partitionLengths = mapOutputWriter.commitAllPartitions(); mapStatus = MapStatus$.MODULE$.apply( - blockManager.shuffleServerId(), partitionLengths, mapTaskAttemptId); + blockManager.shuffleServerId(), partitionLengths, mapId); return; } final SerializerInstance serInstance = serializer.newInstance(); @@ -166,7 +166,7 @@ public void write(Iterator> records) throws IOException { partitionLengths = writePartitionedData(mapOutputWriter); mapStatus = MapStatus$.MODULE$.apply( - blockManager.shuffleServerId(), partitionLengths, mapTaskAttemptId); + blockManager.shuffleServerId(), partitionLengths, mapId); } catch (Exception e) { try { mapOutputWriter.abort(e); diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java index cd82d6abebaa8..f7568ae4a175d 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java @@ -78,7 +78,7 @@ public class UnsafeShuffleWriter extends ShuffleWriter { private final ShuffleWriteMetricsReporter writeMetrics; private final ShuffleExecutorComponents shuffleExecutorComponents; private final int shuffleId; - private final long mapTaskAttemptId; + private final long mapId; private final TaskContext taskContext; private final SparkConf sparkConf; private final boolean transferToEnabled; @@ -122,7 +122,7 @@ public UnsafeShuffleWriter( } this.blockManager = blockManager; this.memoryManager = memoryManager; - this.mapTaskAttemptId = taskContext.taskAttemptId(); + this.mapId = taskContext.taskAttemptId(); final ShuffleDependency dep = handle.dependency(); this.shuffleId = dep.shuffleId(); this.serializer = dep.serializer().newInstance(); @@ -227,8 +227,7 @@ void closeAndWriteOutput() throws IOException { } } } - mapStatus = MapStatus$.MODULE$.apply( - blockManager.shuffleServerId(), partitionLengths, mapTaskAttemptId); + mapStatus = MapStatus$.MODULE$.apply(blockManager.shuffleServerId(), partitionLengths, mapId); } @VisibleForTesting diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleExecutorComponents.java b/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleExecutorComponents.java index d3e8c2e4dea1c..a0c7d3c248d48 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleExecutorComponents.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleExecutorComponents.java @@ -61,14 +61,14 @@ public void initializeExecutor(String appId, String execId) { @Override public ShuffleMapOutputWriter createMapOutputWriter( int shuffleId, - long mapTaskAttemptId, + long mapId, int numPartitions) { if (blockResolver == null) { throw new IllegalStateException( "Executor components must be initialized before getting writers."); } return new LocalDiskShuffleMapOutputWriter( - shuffleId, mapTaskAttemptId, numPartitions, blockResolver, sparkConf); + shuffleId, mapId, numPartitions, blockResolver, sparkConf); } @Override diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriter.java index cdf41b4401d7a..a6529fd76188a 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriter.java @@ -48,7 +48,7 @@ public class LocalDiskShuffleMapOutputWriter implements ShuffleMapOutputWriter { LoggerFactory.getLogger(LocalDiskShuffleMapOutputWriter.class); private final int shuffleId; - private final long mapTaskAttemptId; + private final long mapId; private final IndexShuffleBlockResolver blockResolver; private final long[] partitionLengths; private final int bufferSize; @@ -64,18 +64,18 @@ public class LocalDiskShuffleMapOutputWriter implements ShuffleMapOutputWriter { public LocalDiskShuffleMapOutputWriter( int shuffleId, - long mapTaskAttemptId, + long mapId, int numPartitions, IndexShuffleBlockResolver blockResolver, SparkConf sparkConf) { this.shuffleId = shuffleId; - this.mapTaskAttemptId = mapTaskAttemptId; + this.mapId = mapId; this.blockResolver = blockResolver; this.bufferSize = (int) (long) sparkConf.get( package$.MODULE$.SHUFFLE_UNSAFE_FILE_OUTPUT_BUFFER_SIZE()) * 1024; this.partitionLengths = new long[numPartitions]; - this.outputFile = blockResolver.getDataFile(shuffleId, mapTaskAttemptId); + this.outputFile = blockResolver.getDataFile(shuffleId, mapId); this.outputTempFile = null; } @@ -112,8 +112,7 @@ public long[] commitAllPartitions() throws IOException { } cleanUp(); File resolvedTmp = outputTempFile != null && outputTempFile.isFile() ? outputTempFile : null; - blockResolver.writeIndexFileAndCommit( - shuffleId, mapTaskAttemptId, partitionLengths, resolvedTmp); + blockResolver.writeIndexFileAndCommit(shuffleId, mapId, partitionLengths, resolvedTmp); return partitionLengths; } diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 6e805561cc532..b1575ec92cea5 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -283,7 +283,7 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging // For testing def getMapSizesByExecutorId(shuffleId: Int, reduceId: Int) : Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])] = { - getMapSizesByExecutorId(shuffleId, reduceId, reduceId + 1) + getMapSizesByExecutorId(shuffleId, reduceId, reduceId + 1, useOldFetchProtocol = false) } /** @@ -295,8 +295,12 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging * and the second item is a sequence of (shuffle block id, shuffle block size, map id) * tuples describing the shuffle blocks that are stored at that block manager. */ - def getMapSizesByExecutorId(shuffleId: Int, startPartition: Int, endPartition: Int) - : Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])] + def getMapSizesByExecutorId( + shuffleId: Int, + startPartition: Int, + endPartition: Int, + useOldFetchProtocol: Boolean) + : Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])] /** * Deletes map output status information for the specified shuffle stage. @@ -645,13 +649,18 @@ private[spark] class MapOutputTrackerMaster( // Get blocks sizes by executor Id. Note that zero-sized blocks are excluded in the result. // This method is only called in local-mode. - def getMapSizesByExecutorId(shuffleId: Int, startPartition: Int, endPartition: Int) - : Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])] = { + def getMapSizesByExecutorId( + shuffleId: Int, + startPartition: Int, + endPartition: Int, + useOldFetchProtocol: Boolean) + : Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])] = { logDebug(s"Fetching outputs for shuffle $shuffleId, partitions $startPartition-$endPartition") shuffleStatuses.get(shuffleId) match { case Some (shuffleStatus) => shuffleStatus.withMapStatuses { statuses => - MapOutputTracker.convertMapStatuses(shuffleId, startPartition, endPartition, statuses) + MapOutputTracker.convertMapStatuses( + shuffleId, startPartition, endPartition, statuses, useOldFetchProtocol) } case None => Iterator.empty @@ -685,12 +694,17 @@ private[spark] class MapOutputTrackerWorker(conf: SparkConf) extends MapOutputTr private val fetchingLock = new KeyLock[Int] // Get blocks sizes by executor Id. Note that zero-sized blocks are excluded in the result. - override def getMapSizesByExecutorId(shuffleId: Int, startPartition: Int, endPartition: Int) - : Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])] = { + override def getMapSizesByExecutorId( + shuffleId: Int, + startPartition: Int, + endPartition: Int, + useOldFetchProtocol: Boolean) + : Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])] = { logDebug(s"Fetching outputs for shuffle $shuffleId, partitions $startPartition-$endPartition") val statuses = getStatuses(shuffleId) try { - MapOutputTracker.convertMapStatuses(shuffleId, startPartition, endPartition, statuses) + MapOutputTracker.convertMapStatuses( + shuffleId, startPartition, endPartition, statuses, useOldFetchProtocol) } catch { case e: MetadataFetchFailedException => // We experienced a fetch failure so our mapStatuses cache is outdated; clear it: @@ -833,18 +847,20 @@ private[spark] object MapOutputTracker extends Logging { * @param startPartition Start of map output partition ID range (included in range) * @param endPartition End of map output partition ID range (excluded from range) * @param statuses List of map statuses, indexed by map ID. + * @param useOldFetchProtocol Whether to use the old shuffle fetch protocol. * @return A sequence of 2-item tuples, where the first item in the tuple is a BlockManagerId, - * and the second item is a sequence of (shuffle block id, shuffle block size, map id) + * and the second item is a sequence of (shuffle block id, shuffle block size, map index) * tuples describing the shuffle blocks that are stored at that block manager. */ def convertMapStatuses( shuffleId: Int, startPartition: Int, endPartition: Int, - statuses: Array[MapStatus]): Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])] = { + statuses: Array[MapStatus], + useOldFetchProtocol: Boolean): Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])] = { assert (statuses != null) val splitsByAddress = new HashMap[BlockManagerId, ListBuffer[(BlockId, Long, Int)]] - for ((status, mapId) <- statuses.iterator.zipWithIndex) { + for ((status, mapIndex) <- statuses.iterator.zipWithIndex) { if (status == null) { val errorMessage = s"Missing an output location for shuffle $shuffleId" logError(errorMessage) @@ -853,8 +869,15 @@ private[spark] object MapOutputTracker extends Logging { for (part <- startPartition until endPartition) { val size = status.getSizeForBlock(part) if (size != 0) { - splitsByAddress.getOrElseUpdate(status.location, ListBuffer()) += - ((ShuffleBlockId(shuffleId, status.mapTaskAttemptId, part), size, mapId)) + if (useOldFetchProtocol) { + // While we use the old shuffle fetch protocol, we use mapIndex as mapId in the + // ShuffleBlockId, so here set mapIndex with a invalid value -1. + splitsByAddress.getOrElseUpdate(status.location, ListBuffer()) += + ((ShuffleBlockId(shuffleId, mapIndex, part), size, -1)) + } else { + splitsByAddress.getOrElseUpdate(status.location, ListBuffer()) += + ((ShuffleBlockId(shuffleId, status.mapId, part), size, mapIndex)) + } } } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala b/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala index f91fe608d80de..034cb38b14cd1 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala @@ -47,7 +47,7 @@ private[spark] sealed trait MapStatus { /** * The unique ID of this shuffle map task, we use taskContext.taskAttemptId to fill this. */ - def mapTaskAttemptId: Long + def mapId: Long } @@ -64,11 +64,11 @@ private[spark] object MapStatus { def apply( loc: BlockManagerId, uncompressedSizes: Array[Long], - mapTaskAttemptId: Long): MapStatus = { + mapId: Long): MapStatus = { if (uncompressedSizes.length > minPartitionsToUseHighlyCompressMapStatus) { - HighlyCompressedMapStatus(loc, uncompressedSizes, mapTaskAttemptId) + HighlyCompressedMapStatus(loc, uncompressedSizes, mapId) } else { - new CompressedMapStatus(loc, uncompressedSizes, mapTaskAttemptId) + new CompressedMapStatus(loc, uncompressedSizes, mapId) } } @@ -108,11 +108,12 @@ private[spark] object MapStatus { * * @param loc location where the task is being executed. * @param compressedSizes size of the blocks, indexed by reduce partition id. + * @param mapTaskId unique task id for the task */ private[spark] class CompressedMapStatus( private[this] var loc: BlockManagerId, private[this] var compressedSizes: Array[Byte], - private[this] var taskAttemptId: Long) + private[this] var mapTaskId: Long) extends MapStatus with Externalizable { // For deserialization only @@ -128,13 +129,13 @@ private[spark] class CompressedMapStatus( MapStatus.decompressSize(compressedSizes(reduceId)) } - override def mapTaskAttemptId: Long = taskAttemptId + override def mapId: Long = mapTaskId override def writeExternal(out: ObjectOutput): Unit = Utils.tryOrIOException { loc.writeExternal(out) out.writeInt(compressedSizes.length) out.write(compressedSizes) - out.writeLong(taskAttemptId) + out.writeLong(mapTaskId) } override def readExternal(in: ObjectInput): Unit = Utils.tryOrIOException { @@ -142,7 +143,7 @@ private[spark] class CompressedMapStatus( val len = in.readInt() compressedSizes = new Array[Byte](len) in.readFully(compressedSizes) - taskAttemptId = in.readLong() + mapTaskId = in.readLong() } } @@ -156,7 +157,7 @@ private[spark] class CompressedMapStatus( * @param emptyBlocks a bitmap tracking which blocks are empty * @param avgSize average size of the non-empty and non-huge blocks * @param hugeBlockSizes sizes of huge blocks by their reduceId. - * @param taskAttemptId unique task id for the task + * @param mapTaskId unique task id for the task */ private[spark] class HighlyCompressedMapStatus private ( private[this] var loc: BlockManagerId, @@ -164,12 +165,12 @@ private[spark] class HighlyCompressedMapStatus private ( private[this] var emptyBlocks: RoaringBitmap, private[this] var avgSize: Long, private[this] var hugeBlockSizes: scala.collection.Map[Int, Byte], - private[this] var taskAttemptId: Long) + private[this] var mapTaskId: Long) extends MapStatus with Externalizable { // loc could be null when the default constructor is called during deserialization require(loc == null || avgSize > 0 || hugeBlockSizes.size > 0 - || numNonEmptyBlocks == 0 || taskAttemptId > 0, + || numNonEmptyBlocks == 0 || mapTaskId > 0, "Average size can only be zero for map stages that produced no output") protected def this() = this(null, -1, null, -1, null, -1) // For deserialization only @@ -188,7 +189,7 @@ private[spark] class HighlyCompressedMapStatus private ( } } - override def mapTaskAttemptId: Long = taskAttemptId + override def mapId: Long = mapTaskId override def writeExternal(out: ObjectOutput): Unit = Utils.tryOrIOException { loc.writeExternal(out) @@ -199,7 +200,7 @@ private[spark] class HighlyCompressedMapStatus private ( out.writeInt(kv._1) out.writeByte(kv._2) } - out.writeLong(taskAttemptId) + out.writeLong(mapTaskId) } override def readExternal(in: ObjectInput): Unit = Utils.tryOrIOException { @@ -215,7 +216,7 @@ private[spark] class HighlyCompressedMapStatus private ( hugeBlockSizesImpl(block) = size } hugeBlockSizes = hugeBlockSizesImpl - taskAttemptId = in.readLong() + mapTaskId = in.readLong() } } @@ -223,7 +224,7 @@ private[spark] object HighlyCompressedMapStatus { def apply( loc: BlockManagerId, uncompressedSizes: Array[Long], - mapTaskAttemptId: Long): HighlyCompressedMapStatus = { + mapId: Long): HighlyCompressedMapStatus = { // We must keep track of which blocks are empty so that we don't report a zero-sized // block as being non-empty (or vice-versa) when using the average block size. var i = 0 @@ -264,6 +265,6 @@ private[spark] object HighlyCompressedMapStatus { emptyBlocks.trim() emptyBlocks.runOptimize() new HighlyCompressedMapStatus(loc, numNonEmptyBlocks, emptyBlocks, avgSize, - hugeBlockSizes, mapTaskAttemptId) + hugeBlockSizes, mapId) } } diff --git a/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala b/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala index 4329824b1b627..8a0e84d901c2f 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala @@ -47,7 +47,8 @@ private[spark] class BlockStoreShuffleReader[K, C]( context, blockManager.blockStoreClient, blockManager, - mapOutputTracker.getMapSizesByExecutorId(handle.shuffleId, startPartition, endPartition), + mapOutputTracker.getMapSizesByExecutorId(handle.shuffleId, startPartition, endPartition, + SparkEnv.get.conf.get(config.SHUFFLE_USE_OLD_FETCH_PROTOCOL)), serializerManager.wrapStream, // Note: we use getSizeAsMb when no suffix is provided for backwards compatibility SparkEnv.get.conf.get(config.REDUCER_MAX_SIZE_IN_FLIGHT) * 1024 * 1024, diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala index 264cc2aa9f901..acb09ef3b73f0 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala @@ -63,12 +63,12 @@ private[spark] class SortShuffleWriter[K, V, C]( // Don't bother including the time to open the merged output file in the shuffle write time, // because it just opens a single file, so is typically too fast to measure accurately // (see SPARK-3570). - val mapTaskAttemptId = context.taskAttemptId() + val mapId = context.taskAttemptId() val mapOutputWriter = shuffleExecutorComponents.createMapOutputWriter( - dep.shuffleId, mapTaskAttemptId, dep.partitioner.numPartitions) - sorter.writePartitionedMapOutput(dep.shuffleId, mapTaskAttemptId, mapOutputWriter) + dep.shuffleId, mapId, dep.partitioner.numPartitions) + sorter.writePartitionedMapOutput(dep.shuffleId, mapId, mapOutputWriter) val partitionLengths = mapOutputWriter.commitAllPartitions() - mapStatus = MapStatus(blockManager.shuffleServerId, partitionLengths, mapTaskAttemptId) + mapStatus = MapStatus(blockManager.shuffleServerId, partitionLengths, mapId) } /** Close this writer, passing along whether the map completed */ diff --git a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala index d5ee19bde8edf..da2ba2165bb0c 100644 --- a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala @@ -317,7 +317,7 @@ class MapOutputTrackerSuite extends SparkFunSuite { tracker.registerMapOutput(10, 1, MapStatus(BlockManagerId("b", "hostB", 1000), Array(size10000, size0, size1000, size0), 6)) assert(tracker.containsShuffle(10)) - assert(tracker.getMapSizesByExecutorId(10, 0, 4).toSeq === + assert(tracker.getMapSizesByExecutorId(10, 0, 4, false).toSeq === Seq( (BlockManagerId("a", "hostA", 1000), Seq((ShuffleBlockId(10, 5, 1), size1000, 0), diff --git a/core/src/test/scala/org/apache/spark/memory/MemoryTestingUtils.scala b/core/src/test/scala/org/apache/spark/memory/MemoryTestingUtils.scala index 72708bc41a533..078392279fa05 100644 --- a/core/src/test/scala/org/apache/spark/memory/MemoryTestingUtils.scala +++ b/core/src/test/scala/org/apache/spark/memory/MemoryTestingUtils.scala @@ -25,13 +25,13 @@ import org.apache.spark.{SparkEnv, TaskContext, TaskContextImpl} * Helper methods for mocking out memory-management-related classes in tests. */ object MemoryTestingUtils { - def fakeTaskContext(env: SparkEnv, taskAttemptId: Long = 0): TaskContext = { + def fakeTaskContext(env: SparkEnv, mapTaskId: Long = 0): TaskContext = { val taskMemoryManager = new TaskMemoryManager(env.memoryManager, 0) new TaskContextImpl( stageId = 0, stageAttemptNumber = 0, partitionId = 0, - taskAttemptId = taskAttemptId, + taskAttemptId = mapTaskId, attemptNumber = 0, taskMemoryManager = taskMemoryManager, localProperties = new Properties, diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index cded7061d566a..ecd960ed7fbbe 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -490,7 +490,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi MapStatus(BlockManagerId("exec-hostA1", "hostA", 12345), Array.fill[Long](1)(2), 5)), (Success, MapStatus(BlockManagerId("exec-hostA2", "hostA", 12345), Array.fill[Long](1)(2), 6)), - (Success, makeMapStatus("hostB", 1, taskAttemptId = 7)) + (Success, makeMapStatus("hostB", 1, mapTaskId = 7)) )) // map stage2 completes successfully, with one task on each executor complete(taskSets(1), Seq( @@ -498,7 +498,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi MapStatus(BlockManagerId("exec-hostA1", "hostA", 12345), Array.fill[Long](1)(2), 8)), (Success, MapStatus(BlockManagerId("exec-hostA2", "hostA", 12345), Array.fill[Long](1)(2), 9)), - (Success, makeMapStatus("hostB", 1, taskAttemptId = 10)) + (Success, makeMapStatus("hostB", 1, mapTaskId = 10)) )) // make sure our test setup is correct val initialMapStatus1 = mapOutputTracker.shuffleStatuses(firstShuffleId).mapStatuses @@ -506,14 +506,14 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi assert(initialMapStatus1.count(_ != null) === 3) assert(initialMapStatus1.map{_.location.executorId}.toSet === Set("exec-hostA1", "exec-hostA2", "exec-hostB")) - assert(initialMapStatus1.map{_.mapTaskAttemptId}.toSet === Set(5, 6, 7)) + assert(initialMapStatus1.map{_.mapId}.toSet === Set(5, 6, 7)) val initialMapStatus2 = mapOutputTracker.shuffleStatuses(secondShuffleId).mapStatuses // val initialMapStatus1 = mapOutputTracker.mapStatuses.get(0).get assert(initialMapStatus2.count(_ != null) === 3) assert(initialMapStatus2.map{_.location.executorId}.toSet === Set("exec-hostA1", "exec-hostA2", "exec-hostB")) - assert(initialMapStatus2.map{_.mapTaskAttemptId}.toSet === Set(8, 9, 10)) + assert(initialMapStatus2.map{_.mapId}.toSet === Set(8, 9, 10)) // reduce stage fails with a fetch failure from one host complete(taskSets(2), Seq( @@ -3137,9 +3137,8 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi } object DAGSchedulerSuite { - def makeMapStatus(host: String, reduces: Int, - sizes: Byte = 2, taskAttemptId: Long = -1): MapStatus = - MapStatus(makeBlockManagerId(host), Array.fill[Long](reduces)(sizes), taskAttemptId) + def makeMapStatus(host: String, reduces: Int, sizes: Byte = 2, mapTaskId: Long = -1): MapStatus = + MapStatus(makeBlockManagerId(host), Array.fill[Long](reduces)(sizes), mapTaskId) def makeBlockManagerId(host: String): BlockManagerId = BlockManagerId("exec-" + host, host, 12345) diff --git a/core/src/test/scala/org/apache/spark/scheduler/MapStatusSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/MapStatusSuite.scala index 886535ba4aa97..23cc416f8572f 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/MapStatusSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/MapStatusSuite.scala @@ -92,7 +92,7 @@ class MapStatusSuite extends SparkFunSuite { val status1 = compressAndDecompressMapStatus(status) assert(status1.isInstanceOf[HighlyCompressedMapStatus]) assert(status1.location == loc) - assert(status1.mapTaskAttemptId == mapTaskAttemptId) + assert(status1.mapId == mapTaskAttemptId) for (i <- 0 until 3000) { val estimate = status1.getSizeForBlock(i) if (sizes(i) > 0) { diff --git a/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderSuite.scala index 1a576d82a0a08..76fb5d224e542 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderSuite.scala @@ -102,7 +102,8 @@ class BlockStoreShuffleReaderSuite extends SparkFunSuite with LocalSparkContext // Make a mocked MapOutputTracker for the shuffle reader to use to determine what // shuffle data to read. val mapOutputTracker = mock(classOf[MapOutputTracker]) - when(mapOutputTracker.getMapSizesByExecutorId(shuffleId, reduceId, reduceId + 1)).thenReturn { + when(mapOutputTracker.getMapSizesByExecutorId( + shuffleId, reduceId, reduceId + 1, false)).thenReturn { // Test a scenario where all data is local, to avoid creating a bunch of additional mocks // for the code to read data over the network. val shuffleBlockIdsAndSizes = (0 until numMaps).map { mapId => diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleWriterSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleWriterSuite.scala index e452b1a2a8f43..0923b5ffd5e59 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleWriterSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleWriterSuite.scala @@ -72,7 +72,7 @@ class SortShuffleWriterSuite extends SparkFunSuite with SharedSparkContext with } test("write empty iterator") { - val context = MemoryTestingUtils.fakeTaskContext(sc.env, taskAttemptId = 1) + val context = MemoryTestingUtils.fakeTaskContext(sc.env, mapTaskId = 1) val writer = new SortShuffleWriter[Int, Int, Int]( shuffleBlockResolver, shuffleHandle, @@ -88,7 +88,7 @@ class SortShuffleWriterSuite extends SparkFunSuite with SharedSparkContext with } test("write with some records") { - val context = MemoryTestingUtils.fakeTaskContext(sc.env, taskAttemptId = 2) + val context = MemoryTestingUtils.fakeTaskContext(sc.env, mapTaskId = 2) val records = List[(Int, Int)]((1, 2), (2, 3), (4, 4), (6, 5)) val writer = new SortShuffleWriter[Int, Int, Int]( shuffleBlockResolver, From b31d1f5ddf23163a7259b0da71e13a51321656e4 Mon Sep 17 00:00:00 2001 From: Yuanjian Li Date: Sat, 31 Aug 2019 20:51:21 +0800 Subject: [PATCH 05/15] comment address --- .../api/ShuffleExecutorComponents.java | 4 +- .../shuffle/sort/SortShuffleManager.scala | 7 ++- .../storage/ShuffleBlockFetcherIterator.scala | 50 +++++++++---------- 3 files changed, 29 insertions(+), 32 deletions(-) diff --git a/core/src/main/java/org/apache/spark/shuffle/api/ShuffleExecutorComponents.java b/core/src/main/java/org/apache/spark/shuffle/api/ShuffleExecutorComponents.java index adbb1b773bf82..4d845cdd4a8f2 100644 --- a/core/src/main/java/org/apache/spark/shuffle/api/ShuffleExecutorComponents.java +++ b/core/src/main/java/org/apache/spark/shuffle/api/ShuffleExecutorComponents.java @@ -41,9 +41,7 @@ public interface ShuffleExecutorComponents { * Called once per map task to create a writer that will be responsible for persisting all the * partitioned bytes written by that map task. * @param shuffleId Unique identifier for the shuffle the map task is a part of - * @param mapId Identifier of the task attempt. Multiple attempts of the same map task with the - * same (shuffleId, mapIndex) pair can be distinguished by the different values of - * mapId. + * @param mapId An ID of the map task. The ID is unique within this Spark application. * @param numPartitions The number of partitions that will be written by the map task. Some of * these partitions may be empty. */ diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala index 32b0536abba3a..7d21f32aeca6c 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala @@ -133,10 +133,9 @@ private[spark] class SortShuffleManager(conf: SparkConf) extends ShuffleManager handle: ShuffleHandle, context: TaskContext, metrics: ShuffleWriteMetricsReporter): ShuffleWriter[K, V] = { - taskIdMapsForShuffle.synchronized { - taskIdMapsForShuffle.putIfAbsent(handle.shuffleId, ArrayBuffer.empty[Long]) - taskIdMapsForShuffle.get(handle.shuffleId).append(context.taskAttemptId()) - } + val mapTaskIds = taskIdMapsForShuffle.computeIfAbsent( + handle.shuffleId, _ => ArrayBuffer.empty[Long]) + mapTaskIds.synchronized { mapTaskIds.append(context.taskAttemptId()) } val env = SparkEnv.get handle match { case unsafeShuffleHandle: SerializedShuffleHandle[K @unchecked, V @unchecked] => diff --git a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala index e45962f7ffebf..6b2bdc9e6a67e 100644 --- a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala @@ -50,8 +50,8 @@ import org.apache.spark.util.{CompletionIterator, TaskCompletionListener, Utils} * @param blockManager [[BlockManager]] for reading local blocks * @param blocksByAddress list of blocks to fetch grouped by the [[BlockManagerId]]. * For each block we also require two info: 1. the size (in bytes as a long - * field) in order to throttle the memory usage; 2. the mapId for this - * block, which indicate the index in the map stage of the block. + * field) in order to throttle the memory usage; 2. the mapIndex for this + * block, which indicate the index in the map stage. * Note that zero-sized blocks are already excluded, which happened in * [[org.apache.spark.MapOutputTracker.convertMapStatuses]]. * @param streamWrapper A function to wrap the returned input stream. @@ -344,13 +344,13 @@ final class ShuffleBlockFetcherIterator( logDebug(s"Start fetching local blocks: ${localBlocks.mkString(", ")}") val iter = localBlocks.iterator while (iter.hasNext) { - val (blockId, mapId) = iter.next() + val (blockId, mapIndex) = iter.next() try { val buf = blockManager.getBlockData(blockId) shuffleMetrics.incLocalBlocksFetched(1) shuffleMetrics.incLocalBytesRead(buf.size) buf.retain() - results.put(new SuccessFetchResult(blockId, mapId, blockManager.blockManagerId, + results.put(new SuccessFetchResult(blockId, mapIndex, blockManager.blockManagerId, buf.size(), buf, false)) } catch { // If we see an exception, stop immediately. @@ -363,7 +363,7 @@ final class ShuffleBlockFetcherIterator( logError("Error occurred while fetching local blocks, " + ce.getMessage) case ex: Exception => logError("Error occurred while fetching local blocks", ex) } - results.put(new FailureFetchResult(blockId, mapId, blockManager.blockManagerId, e)) + results.put(new FailureFetchResult(blockId, mapIndex, blockManager.blockManagerId, e)) return } } @@ -423,7 +423,7 @@ final class ShuffleBlockFetcherIterator( shuffleMetrics.incFetchWaitTime(fetchWaitTime) result match { - case r @ SuccessFetchResult(blockId, mapId, address, size, buf, isNetworkReqDone) => + case r @ SuccessFetchResult(blockId, mapIndex, address, size, buf, isNetworkReqDone) => if (address != blockManager.blockManagerId) { numBlocksInFlightPerAddress(address) = numBlocksInFlightPerAddress(address) - 1 shuffleMetrics.incRemoteBytesRead(buf.size) @@ -432,7 +432,7 @@ final class ShuffleBlockFetcherIterator( } shuffleMetrics.incRemoteBlocksFetched(1) } - if (!localBlocks.contains((blockId, mapId))) { + if (!localBlocks.contains((blockId, mapIndex))) { bytesInFlight -= size } if (isNetworkReqDone) { @@ -456,7 +456,7 @@ final class ShuffleBlockFetcherIterator( // since the last call. val msg = s"Received a zero-size buffer for block $blockId from $address " + s"(expectedApproxSize = $size, isNetworkReqDone=$isNetworkReqDone)" - throwFetchFailedException(blockId, mapId, address, new IOException(msg)) + throwFetchFailedException(blockId, mapIndex, address, new IOException(msg)) } val in = try { @@ -472,7 +472,7 @@ final class ShuffleBlockFetcherIterator( case e: IOException => logError("Failed to create input stream from local block", e) } buf.release() - throwFetchFailedException(blockId, mapId, address, e) + throwFetchFailedException(blockId, mapIndex, address, e) } try { input = streamWrapper(blockId, in) @@ -490,11 +490,11 @@ final class ShuffleBlockFetcherIterator( buf.release() if (buf.isInstanceOf[FileSegmentManagedBuffer] || corruptedBlocks.contains(blockId)) { - throwFetchFailedException(blockId, mapId, address, e) + throwFetchFailedException(blockId, mapIndex, address, e) } else { logWarning(s"got an corrupted block $blockId from $address, fetch again", e) corruptedBlocks += blockId - fetchRequests += FetchRequest(address, Array((blockId, size, mapId))) + fetchRequests += FetchRequest(address, Array((blockId, size, mapIndex))) result = null } } finally { @@ -506,8 +506,8 @@ final class ShuffleBlockFetcherIterator( } } - case FailureFetchResult(blockId, mapId, address, e) => - throwFetchFailedException(blockId, mapId, address, e) + case FailureFetchResult(blockId, mapIndex, address, e) => + throwFetchFailedException(blockId, mapIndex, address, e) } // Send fetch requests up to maxBytesInFlight @@ -520,7 +520,7 @@ final class ShuffleBlockFetcherIterator( input, this, currentResult.blockId, - currentResult.mapId, + currentResult.mapIndex, currentResult.address, detectCorrupt && streamCompressedOrEncrypted)) } @@ -587,12 +587,12 @@ final class ShuffleBlockFetcherIterator( private[storage] def throwFetchFailedException( blockId: BlockId, - mapId: Int, + mapIndex: Int, address: BlockManagerId, e: Throwable) = { blockId match { case ShuffleBlockId(shufId, _, reduceId) => - throw new FetchFailedException(address, shufId.toInt, mapId.toInt, reduceId, e) + throw new FetchFailedException(address, shufId.toInt, mapIndex.toInt, reduceId, e) case _ => throw new SparkException( "Failed to get block " + blockId + ", which is not a shuffle block", e) @@ -609,7 +609,7 @@ private class BufferReleasingInputStream( private[storage] val delegate: InputStream, private val iterator: ShuffleBlockFetcherIterator, private val blockId: BlockId, - private val mapId: Int, + private val mapIndex: Int, private val address: BlockManagerId, private val detectCorruption: Boolean) extends InputStream { @@ -621,7 +621,7 @@ private class BufferReleasingInputStream( } catch { case e: IOException if detectCorruption => IOUtils.closeQuietly(this) - iterator.throwFetchFailedException(blockId, mapId, address, e) + iterator.throwFetchFailedException(blockId, mapIndex, address, e) } } @@ -643,7 +643,7 @@ private class BufferReleasingInputStream( } catch { case e: IOException if detectCorruption => IOUtils.closeQuietly(this) - iterator.throwFetchFailedException(blockId, mapId, address, e) + iterator.throwFetchFailedException(blockId, mapIndex, address, e) } } @@ -655,7 +655,7 @@ private class BufferReleasingInputStream( } catch { case e: IOException if detectCorruption => IOUtils.closeQuietly(this) - iterator.throwFetchFailedException(blockId, mapId, address, e) + iterator.throwFetchFailedException(blockId, mapIndex, address, e) } } @@ -665,7 +665,7 @@ private class BufferReleasingInputStream( } catch { case e: IOException if detectCorruption => IOUtils.closeQuietly(this) - iterator.throwFetchFailedException(blockId, mapId, address, e) + iterator.throwFetchFailedException(blockId, mapIndex, address, e) } } @@ -718,7 +718,7 @@ object ShuffleBlockFetcherIterator { /** * Result of a fetch from a remote block successfully. * @param blockId block id - * @param mapId mapId for this block + * @param mapIndex the mapIndex for this block, which indicate the index in the map stage. * @param address BlockManager that the block was fetched from. * @param size estimated size of the block. Note that this is NOT the exact bytes. * Size of remote block is used to calculate bytesInFlight. @@ -727,7 +727,7 @@ object ShuffleBlockFetcherIterator { */ private[storage] case class SuccessFetchResult( blockId: BlockId, - mapId: Int, + mapIndex: Int, address: BlockManagerId, size: Long, buf: ManagedBuffer, @@ -739,13 +739,13 @@ object ShuffleBlockFetcherIterator { /** * Result of a fetch from a remote block unsuccessfully. * @param blockId block id - * @param mapId mapId for this block + * @param mapIndex the mapIndex for this block, which indicate the index in the map stage * @param address BlockManager that the block was attempted to be fetched from * @param e the failure exception */ private[storage] case class FailureFetchResult( blockId: BlockId, - mapId: Int, + mapIndex: Int, address: BlockManagerId, e: Throwable) extends FetchResult From ff8fde9643a3e698a93c8332d6b84342f43ed65b Mon Sep 17 00:00:00 2001 From: Yuanjian Li Date: Thu, 5 Sep 2019 20:56:21 +0800 Subject: [PATCH 06/15] rename mapId to mapTaskId and mapIndex, address commments --- .../network/shuffle/ExternalBlockHandler.java | 19 +++++----- .../shuffle/ExternalShuffleBlockResolver.java | 19 +++++----- .../shuffle/OneForOneBlockFetcher.java | 22 ++++++------ .../shuffle/protocol/FetchShuffleBlocks.java | 26 +++++++------- .../api/ShuffleExecutorComponents.java | 4 +-- .../sort/BypassMergeSortShuffleWriter.java | 12 +++---- .../shuffle/sort/UnsafeShuffleWriter.java | 7 ++-- .../LocalDiskShuffleExecutorComponents.java | 4 +-- .../io/LocalDiskShuffleMapOutputWriter.java | 10 +++--- .../org/apache/spark/MapOutputTracker.scala | 36 +++++++++---------- .../org/apache/spark/TaskEndReason.scala | 6 ++-- .../spark/internal/config/package.scala | 2 +- .../network/netty/NettyBlockRpcServer.scala | 11 +++--- .../apache/spark/scheduler/DAGScheduler.scala | 10 +++--- .../apache/spark/scheduler/MapStatus.scala | 34 +++++++++--------- .../org/apache/spark/scheduler/Stage.scala | 2 +- .../spark/shuffle/FetchFailedException.scala | 8 ++--- .../shuffle/IndexShuffleBlockResolver.scala | 25 ++++++------- .../shuffle/sort/SortShuffleManager.scala | 15 ++++---- .../shuffle/sort/SortShuffleWriter.scala | 8 ++--- .../org/apache/spark/storage/BlockId.scala | 24 ++++++------- .../storage/ShuffleBlockFetcherIterator.scala | 10 +++--- .../org/apache/spark/util/JsonProtocol.scala | 2 +- .../util/collection/ExternalSorter.scala | 6 ++-- .../apache/spark/executor/ExecutorSuite.scala | 2 +- .../spark/scheduler/DAGSchedulerSuite.scala | 4 +-- .../spark/scheduler/MapStatusSuite.scala | 2 +- .../serializer/KryoSerializerSuite.scala | 6 ++-- .../BlockStoreShuffleReaderSuite.scala | 10 +++--- .../sort/IndexShuffleBlockResolverSuite.scala | 12 +++---- .../apache/spark/storage/BlockIdSuite.scala | 6 ++-- .../ShuffleBlockFetcherIteratorSuite.scala | 14 ++++---- .../apache/spark/util/JsonProtocolSuite.scala | 2 +- project/MimaExcludes.scala | 9 ++--- 34 files changed, 197 insertions(+), 192 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 19cfac4849b9d..e0a1e263e419e 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 @@ -321,31 +321,32 @@ private class ShuffleManagedBufferIterator implements Iterator { private final String appId; private final String execId; private final int shuffleId; - private final long[] mapIds; + private final long[] mapTaskIds; private final int[][] reduceIds; ShuffleManagedBufferIterator(FetchShuffleBlocks msg) { appId = msg.appId; execId = msg.execId; shuffleId = msg.shuffleId; - mapIds = msg.mapIds; + mapTaskIds = msg.mapTaskIds; reduceIds = msg.reduceIds; } @Override public boolean hasNext() { - // mapIds.length must equal to reduceIds.length, and the passed in FetchShuffleBlocks - // must have non-empty mapIds and reduceIds, see the checking logic in OneForOneBlockFetcher. - return mapIdx < mapIds.length && reduceIdx < reduceIds[mapIdx].length; + // mapTaskIds.length must equal to reduceIds.length, and the passed in FetchShuffleBlocks + // must have non-empty mapTaskIds and reduceIds, see the checking logic in + // OneForOneBlockFetcher. + return mapIdx < mapTaskIds.length && reduceIdx < reduceIds[mapIdx].length; } @Override public ManagedBuffer next() { final ManagedBuffer block = blockManager.getBlockData( - appId, execId, shuffleId, mapIds[mapIdx], reduceIds[mapIdx][reduceIdx]); - reduceIdx += 1; - if (reduceIdx == reduceIds[mapIdx].length) { - // Reach the end of current mapId, move to next mapId and its reduceIds. + appId, execId, shuffleId, mapTaskIds[mapIdx], reduceIds[mapIdx][reduceIdx]); + if (reduceIdx < reduceIds[mapIdx].length - 1) { + reduceIdx += 1; + } else { reduceIdx = 0; mapIdx += 1; } 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 8b0d1e145a813..e51b98fb7c8d3 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 @@ -165,21 +165,21 @@ public void registerExecutor( } /** - * Obtains a FileSegmentManagedBuffer from (shuffleId, mapId, reduceId). We make assumptions + * Obtains a FileSegmentManagedBuffer from (shuffleId, mapTaskId, reduceId). We make assumptions * about how the hash and sort based shuffles store their data. */ public ManagedBuffer getBlockData( String appId, String execId, int shuffleId, - long mapId, + long mapTaskId, int reduceId) { ExecutorShuffleInfo executor = executors.get(new AppExecId(appId, execId)); if (executor == null) { throw new RuntimeException( String.format("Executor is not registered (appId=%s, execId=%s)", appId, execId)); } - return getSortBasedShuffleBlockData(executor, shuffleId, mapId, reduceId); + return getSortBasedShuffleBlockData(executor, shuffleId, mapTaskId, reduceId); } public ManagedBuffer getRddBlockData( @@ -291,14 +291,15 @@ private void deleteNonShuffleServiceServedFiles(String[] dirs) { } /** - * Sort-based shuffle data uses an index called "shuffle_ShuffleId_MapId_0.index" into a data file - * called "shuffle_ShuffleId_MapId_0.data". This logic is from IndexShuffleBlockResolver, - * and the block id format is from ShuffleDataBlockId and ShuffleIndexBlockId. + * Sort-based shuffle data uses an index called "shuffle_ShuffleId_MapTaskId_0.index" into a data + * file called "shuffle_ShuffleId_MapTaskId_0.data". + * This logic is from IndexShuffleBlockResolver, and the block id format is from + * ShuffleDataBlockId and ShuffleIndexBlockId. */ private ManagedBuffer getSortBasedShuffleBlockData( - ExecutorShuffleInfo executor, int shuffleId, long mapId, int reduceId) { + ExecutorShuffleInfo executor, int shuffleId, long mapTaskId, int reduceId) { File indexFile = ExecutorDiskUtils.getFile(executor.localDirs, executor.subDirsPerLocalDir, - "shuffle_" + shuffleId + "_" + mapId + "_0.index"); + "shuffle_" + shuffleId + "_" + mapTaskId + "_0.index"); try { ShuffleIndexInformation shuffleIndexInformation = shuffleIndexCache.get(indexFile); @@ -306,7 +307,7 @@ private ManagedBuffer getSortBasedShuffleBlockData( return new FileSegmentManagedBuffer( conf, ExecutorDiskUtils.getFile(executor.localDirs, executor.subDirsPerLocalDir, - "shuffle_" + shuffleId + "_" + mapId + "_0.data"), + "shuffle_" + shuffleId + "_" + mapTaskId + "_0.data"), shuffleIndexRecord.getOffset(), shuffleIndexRecord.getLength()); } catch (ExecutionException e) { diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java index 52854c86be3e6..446014b775bcf 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java @@ -108,7 +108,7 @@ private boolean isShuffleBlocks(String[] blockIds) { /** * Analyze the pass in blockIds and create FetchShuffleBlocks message. - * The blockIds has been sorted by mapId and reduceId. It's produced in + * The blockIds has been sorted by mapTaskId and reduceId. It's produced in * org.apache.spark.MapOutputTracker.convertMapStatuses. */ private FetchShuffleBlocks createFetchShuffleBlocksMsg( @@ -121,21 +121,21 @@ private FetchShuffleBlocks createFetchShuffleBlocksMsg( throw new IllegalArgumentException("Expected shuffleId=" + shuffleId + ", got:" + blockId); } - long mapId = blockIdParts.middle; - if (!mapIdToReduceIds.containsKey(mapId)) { - mapIdToReduceIds.put(mapId, new ArrayList<>()); + long mapTaskId = blockIdParts.middle; + if (!mapIdToReduceIds.containsKey(mapTaskId)) { + mapIdToReduceIds.put(mapTaskId, new ArrayList<>()); } - mapIdToReduceIds.get(mapId).add(blockIdParts.right); + mapIdToReduceIds.get(mapTaskId).add(blockIdParts.right); } - long[] mapIds = Longs.toArray(mapIdToReduceIds.keySet()); - int[][] reduceIdArr = new int[mapIds.length][]; - for (int i = 0; i < mapIds.length; i++) { - reduceIdArr[i] = Ints.toArray(mapIdToReduceIds.get(mapIds[i])); + long[] mapTaskIds = Longs.toArray(mapIdToReduceIds.keySet()); + int[][] reduceIdArr = new int[mapTaskIds.length][]; + for (int i = 0; i < mapTaskIds.length; i++) { + reduceIdArr[i] = Ints.toArray(mapIdToReduceIds.get(mapTaskIds[i])); } - return new FetchShuffleBlocks(appId, execId, shuffleId, mapIds, reduceIdArr); + return new FetchShuffleBlocks(appId, execId, shuffleId, mapTaskIds, reduceIdArr); } - /** Split the shuffleBlockId and return shuffleId, mapId and reduceId. */ + /** Split the shuffleBlockId and return shuffleId, mapTaskId and reduceId. */ private ImmutableTriple splitBlockId(String blockId) { String[] blockIdParts = blockId.split("_"); if (blockIdParts.length != 4 || !blockIdParts[0].equals("shuffle")) { diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/FetchShuffleBlocks.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/FetchShuffleBlocks.java index faa960d414bcc..63fc47ff9f03c 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/FetchShuffleBlocks.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/FetchShuffleBlocks.java @@ -32,23 +32,23 @@ public class FetchShuffleBlocks extends BlockTransferMessage { public final String appId; public final String execId; public final int shuffleId; - // The length of mapIds must equal to reduceIds.size(), for the i-th mapId in mapIds, - // it corresponds to the i-th int[] in reduceIds, which contains all reduce id for this map id. - public final long[] mapIds; + // The length of mapTaskIds must equal to reduceIds.size(), for the i-th mapTaskId in mapTaskIds, + // it corresponds to the i-th int[] in reduceIds, which contains all reduce id for this map task. + public final long[] mapTaskIds; public final int[][] reduceIds; public FetchShuffleBlocks( String appId, String execId, int shuffleId, - long[] mapIds, + long[] mapTaskIds, int[][] reduceIds) { this.appId = appId; this.execId = execId; this.shuffleId = shuffleId; - this.mapIds = mapIds; + this.mapTaskIds = mapTaskIds; this.reduceIds = reduceIds; - assert(mapIds.length == reduceIds.length); + assert(mapTaskIds.length == reduceIds.length); } @Override @@ -60,7 +60,7 @@ public String toString() { .add("appId", appId) .add("execId", execId) .add("shuffleId", shuffleId) - .add("mapIds", Arrays.toString(mapIds)) + .add("mapTaskIds", Arrays.toString(mapTaskIds)) .add("reduceIds", Arrays.deepToString(reduceIds)) .toString(); } @@ -75,7 +75,7 @@ public boolean equals(Object o) { if (shuffleId != that.shuffleId) return false; if (!appId.equals(that.appId)) return false; if (!execId.equals(that.execId)) return false; - if (!Arrays.equals(mapIds, that.mapIds)) return false; + if (!Arrays.equals(mapTaskIds, that.mapTaskIds)) return false; return Arrays.deepEquals(reduceIds, that.reduceIds); } @@ -84,7 +84,7 @@ public int hashCode() { int result = appId.hashCode(); result = 31 * result + execId.hashCode(); result = 31 * result + shuffleId; - result = 31 * result + Arrays.hashCode(mapIds); + result = 31 * result + Arrays.hashCode(mapTaskIds); result = 31 * result + Arrays.deepHashCode(reduceIds); return result; } @@ -98,7 +98,7 @@ public int encodedLength() { return Encoders.Strings.encodedLength(appId) + Encoders.Strings.encodedLength(execId) + 4 /* encoded length of shuffleId */ - + Encoders.LongArrays.encodedLength(mapIds) + + Encoders.LongArrays.encodedLength(mapTaskIds) + 4 /* encoded length of reduceIds.size() */ + encodedLengthOfReduceIds; } @@ -108,7 +108,7 @@ public void encode(ByteBuf buf) { Encoders.Strings.encode(buf, appId); Encoders.Strings.encode(buf, execId); buf.writeInt(shuffleId); - Encoders.LongArrays.encode(buf, mapIds); + Encoders.LongArrays.encode(buf, mapTaskIds); buf.writeInt(reduceIds.length); for (int[] ids: reduceIds) { Encoders.IntArrays.encode(buf, ids); @@ -119,12 +119,12 @@ public static FetchShuffleBlocks decode(ByteBuf buf) { String appId = Encoders.Strings.decode(buf); String execId = Encoders.Strings.decode(buf); int shuffleId = buf.readInt(); - long[] mapIds = Encoders.LongArrays.decode(buf); + long[] mapTaskIds = Encoders.LongArrays.decode(buf); int reduceIdsSize = buf.readInt(); int[][] reduceIds = new int[reduceIdsSize][]; for (int i = 0; i < reduceIdsSize; i++) { reduceIds[i] = Encoders.IntArrays.decode(buf); } - return new FetchShuffleBlocks(appId, execId, shuffleId, mapIds, reduceIds); + return new FetchShuffleBlocks(appId, execId, shuffleId, mapTaskIds, reduceIds); } } diff --git a/core/src/main/java/org/apache/spark/shuffle/api/ShuffleExecutorComponents.java b/core/src/main/java/org/apache/spark/shuffle/api/ShuffleExecutorComponents.java index 4d845cdd4a8f2..a0fd415cc149c 100644 --- a/core/src/main/java/org/apache/spark/shuffle/api/ShuffleExecutorComponents.java +++ b/core/src/main/java/org/apache/spark/shuffle/api/ShuffleExecutorComponents.java @@ -41,13 +41,13 @@ public interface ShuffleExecutorComponents { * Called once per map task to create a writer that will be responsible for persisting all the * partitioned bytes written by that map task. * @param shuffleId Unique identifier for the shuffle the map task is a part of - * @param mapId An ID of the map task. The ID is unique within this Spark application. + * @param mapTaskId An ID of the map task. The ID is unique within this Spark application. * @param numPartitions The number of partitions that will be written by the map task. Some of * these partitions may be empty. */ ShuffleMapOutputWriter createMapOutputWriter( int shuffleId, - long mapId, + long mapTaskId, int numPartitions) throws IOException; /** diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java index dc157eaa3b253..ae0a8ca05c5b9 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java @@ -85,7 +85,7 @@ final class BypassMergeSortShuffleWriter extends ShuffleWriter { private final Partitioner partitioner; private final ShuffleWriteMetricsReporter writeMetrics; private final int shuffleId; - private final long mapId; + private final long mapTaskId; private final Serializer serializer; private final ShuffleExecutorComponents shuffleExecutorComponents; @@ -105,7 +105,7 @@ final class BypassMergeSortShuffleWriter extends ShuffleWriter { BypassMergeSortShuffleWriter( BlockManager blockManager, BypassMergeSortShuffleHandle handle, - long mapId, + long mapTaskId, SparkConf conf, ShuffleWriteMetricsReporter writeMetrics, ShuffleExecutorComponents shuffleExecutorComponents) { @@ -114,7 +114,7 @@ final class BypassMergeSortShuffleWriter extends ShuffleWriter { this.transferToEnabled = conf.getBoolean("spark.file.transferTo", true); this.blockManager = blockManager; final ShuffleDependency dep = handle.dependency(); - this.mapId = mapId; + this.mapTaskId = mapTaskId; this.shuffleId = dep.shuffleId(); this.partitioner = dep.partitioner(); this.numPartitions = partitioner.numPartitions(); @@ -127,12 +127,12 @@ final class BypassMergeSortShuffleWriter extends ShuffleWriter { public void write(Iterator> records) throws IOException { assert (partitionWriters == null); ShuffleMapOutputWriter mapOutputWriter = shuffleExecutorComponents - .createMapOutputWriter(shuffleId, mapId, numPartitions); + .createMapOutputWriter(shuffleId, mapTaskId, numPartitions); try { if (!records.hasNext()) { partitionLengths = mapOutputWriter.commitAllPartitions(); mapStatus = MapStatus$.MODULE$.apply( - blockManager.shuffleServerId(), partitionLengths, mapId); + blockManager.shuffleServerId(), partitionLengths, mapTaskId); return; } final SerializerInstance serInstance = serializer.newInstance(); @@ -166,7 +166,7 @@ public void write(Iterator> records) throws IOException { partitionLengths = writePartitionedData(mapOutputWriter); mapStatus = MapStatus$.MODULE$.apply( - blockManager.shuffleServerId(), partitionLengths, mapId); + blockManager.shuffleServerId(), partitionLengths, mapTaskId); } catch (Exception e) { try { mapOutputWriter.abort(e); diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java index f7568ae4a175d..2dc14ac2e7f6f 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java @@ -78,7 +78,7 @@ public class UnsafeShuffleWriter extends ShuffleWriter { private final ShuffleWriteMetricsReporter writeMetrics; private final ShuffleExecutorComponents shuffleExecutorComponents; private final int shuffleId; - private final long mapId; + private final long mapTaskId; private final TaskContext taskContext; private final SparkConf sparkConf; private final boolean transferToEnabled; @@ -122,7 +122,7 @@ public UnsafeShuffleWriter( } this.blockManager = blockManager; this.memoryManager = memoryManager; - this.mapId = taskContext.taskAttemptId(); + this.mapTaskId = taskContext.taskAttemptId(); final ShuffleDependency dep = handle.dependency(); this.shuffleId = dep.shuffleId(); this.serializer = dep.serializer().newInstance(); @@ -227,7 +227,8 @@ void closeAndWriteOutput() throws IOException { } } } - mapStatus = MapStatus$.MODULE$.apply(blockManager.shuffleServerId(), partitionLengths, mapId); + mapStatus = MapStatus$.MODULE$.apply( + blockManager.shuffleServerId(), partitionLengths, mapTaskId); } @VisibleForTesting diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleExecutorComponents.java b/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleExecutorComponents.java index a0c7d3c248d48..94b5c9180ff85 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleExecutorComponents.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleExecutorComponents.java @@ -61,14 +61,14 @@ public void initializeExecutor(String appId, String execId) { @Override public ShuffleMapOutputWriter createMapOutputWriter( int shuffleId, - long mapId, + long mapTaskId, int numPartitions) { if (blockResolver == null) { throw new IllegalStateException( "Executor components must be initialized before getting writers."); } return new LocalDiskShuffleMapOutputWriter( - shuffleId, mapId, numPartitions, blockResolver, sparkConf); + shuffleId, mapTaskId, numPartitions, blockResolver, sparkConf); } @Override diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriter.java index a6529fd76188a..559ff915cdfd1 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriter.java @@ -48,7 +48,7 @@ public class LocalDiskShuffleMapOutputWriter implements ShuffleMapOutputWriter { LoggerFactory.getLogger(LocalDiskShuffleMapOutputWriter.class); private final int shuffleId; - private final long mapId; + private final long mapTaskId; private final IndexShuffleBlockResolver blockResolver; private final long[] partitionLengths; private final int bufferSize; @@ -64,18 +64,18 @@ public class LocalDiskShuffleMapOutputWriter implements ShuffleMapOutputWriter { public LocalDiskShuffleMapOutputWriter( int shuffleId, - long mapId, + long mapTaskId, int numPartitions, IndexShuffleBlockResolver blockResolver, SparkConf sparkConf) { this.shuffleId = shuffleId; - this.mapId = mapId; + this.mapTaskId = mapTaskId; this.blockResolver = blockResolver; this.bufferSize = (int) (long) sparkConf.get( package$.MODULE$.SHUFFLE_UNSAFE_FILE_OUTPUT_BUFFER_SIZE()) * 1024; this.partitionLengths = new long[numPartitions]; - this.outputFile = blockResolver.getDataFile(shuffleId, mapId); + this.outputFile = blockResolver.getDataFile(shuffleId, mapTaskId); this.outputTempFile = null; } @@ -112,7 +112,7 @@ public long[] commitAllPartitions() throws IOException { } cleanUp(); File resolvedTmp = outputTempFile != null && outputTempFile.isFile() ? outputTempFile : null; - blockResolver.writeIndexFileAndCommit(shuffleId, mapId, partitionLengths, resolvedTmp); + blockResolver.writeIndexFileAndCommit(shuffleId, mapTaskId, partitionLengths, resolvedTmp); return partitionLengths; } diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index b1575ec92cea5..44b247c10fd63 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -41,7 +41,7 @@ import org.apache.spark.util._ * Helper class used by the [[MapOutputTrackerMaster]] to perform bookkeeping for a single * ShuffleMapStage. * - * This class maintains a mapping from mapIds to `MapStatus`. It also maintains a cache of + * This class maintains a mapping from map index to `MapStatus`. It also maintains a cache of * serialized map statuses in order to speed up tasks' requests for map output statuses. * * All public methods of this class are thread-safe. @@ -88,12 +88,12 @@ private class ShuffleStatus(numPartitions: Int) { * Register a map output. If there is already a registered location for the map output then it * will be replaced by the new location. */ - def addMapOutput(mapId: Int, status: MapStatus): Unit = synchronized { - if (mapStatuses(mapId) == null) { + def addMapOutput(mapIndex: Int, status: MapStatus): Unit = synchronized { + if (mapStatuses(mapIndex) == null) { _numAvailableOutputs += 1 invalidateSerializedMapOutputStatusCache() } - mapStatuses(mapId) = status + mapStatuses(mapIndex) = status } /** @@ -101,10 +101,10 @@ private class ShuffleStatus(numPartitions: Int) { * This is a no-op if there is no registered map output or if the registered output is from a * different block manager. */ - def removeMapOutput(mapId: Int, bmAddress: BlockManagerId): Unit = synchronized { - if (mapStatuses(mapId) != null && mapStatuses(mapId).location == bmAddress) { + def removeMapOutput(mapIndex: Int, bmAddress: BlockManagerId): Unit = synchronized { + if (mapStatuses(mapIndex) != null && mapStatuses(mapIndex).location == bmAddress) { _numAvailableOutputs -= 1 - mapStatuses(mapId) = null + mapStatuses(mapIndex) = null invalidateSerializedMapOutputStatusCache() } } @@ -131,10 +131,10 @@ private class ShuffleStatus(numPartitions: Int) { * remove outputs which are served by an external shuffle server (if one exists). */ def removeOutputsByFilter(f: (BlockManagerId) => Boolean): Unit = synchronized { - for (mapId <- 0 until mapStatuses.length) { - if (mapStatuses(mapId) != null && f(mapStatuses(mapId).location)) { + for (mapIndex <- 0 until mapStatuses.length) { + if (mapStatuses(mapIndex) != null && f(mapStatuses(mapIndex).location)) { _numAvailableOutputs -= 1 - mapStatuses(mapId) = null + mapStatuses(mapIndex) = null invalidateSerializedMapOutputStatusCache() } } @@ -300,7 +300,7 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging startPartition: Int, endPartition: Int, useOldFetchProtocol: Boolean) - : Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])] + : Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])] /** * Deletes map output status information for the specified shuffle stage. @@ -422,15 +422,15 @@ private[spark] class MapOutputTrackerMaster( } } - def registerMapOutput(shuffleId: Int, mapId: Int, status: MapStatus) { - shuffleStatuses(shuffleId).addMapOutput(mapId, status) + def registerMapOutput(shuffleId: Int, mapIndex: Int, status: MapStatus) { + shuffleStatuses(shuffleId).addMapOutput(mapIndex, status) } /** Unregister map output information of the given shuffle, mapper and block manager */ - def unregisterMapOutput(shuffleId: Int, mapId: Int, bmAddress: BlockManagerId) { + def unregisterMapOutput(shuffleId: Int, mapIndex: Int, bmAddress: BlockManagerId) { shuffleStatuses.get(shuffleId) match { case Some(shuffleStatus) => - shuffleStatus.removeMapOutput(mapId, bmAddress) + shuffleStatus.removeMapOutput(mapIndex, bmAddress) incrementEpoch() case None => throw new SparkException("unregisterMapOutput called for nonexistent shuffle ID") @@ -654,7 +654,7 @@ private[spark] class MapOutputTrackerMaster( startPartition: Int, endPartition: Int, useOldFetchProtocol: Boolean) - : Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])] = { + : Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])] = { logDebug(s"Fetching outputs for shuffle $shuffleId, partitions $startPartition-$endPartition") shuffleStatuses.get(shuffleId) match { case Some (shuffleStatus) => @@ -870,13 +870,13 @@ private[spark] object MapOutputTracker extends Logging { val size = status.getSizeForBlock(part) if (size != 0) { if (useOldFetchProtocol) { - // While we use the old shuffle fetch protocol, we use mapIndex as mapId in the + // While we use the old shuffle fetch protocol, we use mapIndex as mapTaskId in the // ShuffleBlockId, so here set mapIndex with a invalid value -1. splitsByAddress.getOrElseUpdate(status.location, ListBuffer()) += ((ShuffleBlockId(shuffleId, mapIndex, part), size, -1)) } else { splitsByAddress.getOrElseUpdate(status.location, ListBuffer()) += - ((ShuffleBlockId(shuffleId, status.mapId, part), size, mapIndex)) + ((ShuffleBlockId(shuffleId, status.mapTaskId, part), size, mapIndex)) } } } diff --git a/core/src/main/scala/org/apache/spark/TaskEndReason.scala b/core/src/main/scala/org/apache/spark/TaskEndReason.scala index 19f71a1dec296..9670d72913f21 100644 --- a/core/src/main/scala/org/apache/spark/TaskEndReason.scala +++ b/core/src/main/scala/org/apache/spark/TaskEndReason.scala @@ -83,14 +83,14 @@ case object Resubmitted extends TaskFailedReason { case class FetchFailed( bmAddress: BlockManagerId, // Note that bmAddress can be null shuffleId: Int, - mapId: Int, + mapIndex: Int, reduceId: Int, message: String) extends TaskFailedReason { override def toErrorString: String = { val bmAddressString = if (bmAddress == null) "null" else bmAddress.toString - s"FetchFailed($bmAddressString, shuffleId=$shuffleId, mapId=$mapId, reduceId=$reduceId, " + - s"message=\n$message\n)" + s"FetchFailed($bmAddressString, shuffleId=$shuffleId, mapIndex=$mapIndex, " + + s"reduceId=$reduceId, message=\n$message\n)" } /** diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index b9693a77d18c5..2155dc6d3aa60 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -1050,7 +1050,7 @@ package object config { private[spark] val SHUFFLE_USE_OLD_FETCH_PROTOCOL = ConfigBuilder("spark.shuffle.useOldFetchProtocol") .doc("Whether to use the old protocol while doing the shuffle block fetching. " + - "It is only enabled while we need the compatibility in the scenario of new spark " + + "It is only enabled while we need the compatibility in the scenario of new Spark " + "version job fetching shuffle blocks from old version external shuffle service.") .booleanConf .createWithDefault(false) diff --git a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala index b2ab31488e4c1..6e807eed503cc 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala +++ b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala @@ -64,11 +64,12 @@ class NettyBlockRpcServer( responseContext.onSuccess(new StreamHandle(streamId, blocksNum).toByteBuffer) case fetchShuffleBlocks: FetchShuffleBlocks => - val blocks = fetchShuffleBlocks.mapIds.zipWithIndex.flatMap { case (mapId, index) => - fetchShuffleBlocks.reduceIds.apply(index).map { reduceId => - blockManager.getBlockData( - ShuffleBlockId(fetchShuffleBlocks.shuffleId, mapId, reduceId)) - } + val blocks = fetchShuffleBlocks.mapTaskIds.zipWithIndex.flatMap { + case (mapTaskId, index) => + fetchShuffleBlocks.reduceIds.apply(index).map { reduceId => + blockManager.getBlockData( + ShuffleBlockId(fetchShuffleBlocks.shuffleId, mapTaskId, reduceId)) + } } val numBlockIds = fetchShuffleBlocks.reduceIds.map(_.length).sum val streamId = streamManager.registerStream(appId, blocks.iterator.asJava, diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 9fb098486fb2d..634b2f802c908 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1107,7 +1107,7 @@ private[spark] class DAGScheduler( // The operation here can make sure for the intermediate stage, `findMissingPartitions()` // returns all partitions every time. stage match { - case sms: ShuffleMapStage if stage.isIndeterminate() => + case sms: ShuffleMapStage if stage.isIndeterminate => mapOutputTracker.unregisterAllMapOutput(sms.shuffleDep.shuffleId) case _ => } @@ -1507,7 +1507,7 @@ private[spark] class DAGScheduler( } } - case FetchFailed(bmAddress, shuffleId, mapId, _, failureMessage) => + case FetchFailed(bmAddress, shuffleId, mapIndex, _, failureMessage) => val failedStage = stageIdToStage(task.stageId) val mapStage = shuffleIdToMapStage(shuffleId) @@ -1538,9 +1538,9 @@ private[spark] class DAGScheduler( // Mark all the map as broken in the map stage, to ensure retry all the tasks on // resubmitted stage attempt. mapOutputTracker.unregisterAllMapOutput(shuffleId) - } else if (mapId != -1) { + } else if (mapIndex != -1) { // Mark the map whose fetch failed as broken in the map stage - mapOutputTracker.unregisterMapOutput(shuffleId, mapId, bmAddress) + mapOutputTracker.unregisterMapOutput(shuffleId, mapIndex, bmAddress) } if (failedStage.rdd.isBarrier()) { @@ -1582,7 +1582,7 @@ private[spark] class DAGScheduler( // Note that, if map stage is UNORDERED, we are fine. The shuffle partitioner is // guaranteed to be determinate, so the input data of the reducers will not change // even if the map tasks are re-tried. - if (mapStage.isIndeterminate()) { + if (mapStage.isIndeterminate) { // It's a little tricky to find all the succeeding stages of `mapStage`, because // each stage only know its parents not children. Here we traverse the stages from // the leaf nodes (the result stages of active jobs), and rollback all the stages diff --git a/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala b/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala index 034cb38b14cd1..75dc8debdd1a6 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala @@ -47,7 +47,7 @@ private[spark] sealed trait MapStatus { /** * The unique ID of this shuffle map task, we use taskContext.taskAttemptId to fill this. */ - def mapId: Long + def mapTaskId: Long } @@ -64,11 +64,11 @@ private[spark] object MapStatus { def apply( loc: BlockManagerId, uncompressedSizes: Array[Long], - mapId: Long): MapStatus = { + mapTaskId: Long): MapStatus = { if (uncompressedSizes.length > minPartitionsToUseHighlyCompressMapStatus) { - HighlyCompressedMapStatus(loc, uncompressedSizes, mapId) + HighlyCompressedMapStatus(loc, uncompressedSizes, mapTaskId) } else { - new CompressedMapStatus(loc, uncompressedSizes, mapId) + new CompressedMapStatus(loc, uncompressedSizes, mapTaskId) } } @@ -108,12 +108,12 @@ private[spark] object MapStatus { * * @param loc location where the task is being executed. * @param compressedSizes size of the blocks, indexed by reduce partition id. - * @param mapTaskId unique task id for the task + * @param mapTId unique task id for the task */ private[spark] class CompressedMapStatus( private[this] var loc: BlockManagerId, private[this] var compressedSizes: Array[Byte], - private[this] var mapTaskId: Long) + private[this] var mapTId: Long) extends MapStatus with Externalizable { // For deserialization only @@ -129,13 +129,13 @@ private[spark] class CompressedMapStatus( MapStatus.decompressSize(compressedSizes(reduceId)) } - override def mapId: Long = mapTaskId + override def mapTaskId: Long = mapTId override def writeExternal(out: ObjectOutput): Unit = Utils.tryOrIOException { loc.writeExternal(out) out.writeInt(compressedSizes.length) out.write(compressedSizes) - out.writeLong(mapTaskId) + out.writeLong(mapTId) } override def readExternal(in: ObjectInput): Unit = Utils.tryOrIOException { @@ -143,7 +143,7 @@ private[spark] class CompressedMapStatus( val len = in.readInt() compressedSizes = new Array[Byte](len) in.readFully(compressedSizes) - mapTaskId = in.readLong() + mapTId = in.readLong() } } @@ -157,7 +157,7 @@ private[spark] class CompressedMapStatus( * @param emptyBlocks a bitmap tracking which blocks are empty * @param avgSize average size of the non-empty and non-huge blocks * @param hugeBlockSizes sizes of huge blocks by their reduceId. - * @param mapTaskId unique task id for the task + * @param mapTId unique task id for the task */ private[spark] class HighlyCompressedMapStatus private ( private[this] var loc: BlockManagerId, @@ -165,12 +165,12 @@ private[spark] class HighlyCompressedMapStatus private ( private[this] var emptyBlocks: RoaringBitmap, private[this] var avgSize: Long, private[this] var hugeBlockSizes: scala.collection.Map[Int, Byte], - private[this] var mapTaskId: Long) + private[this] var mapTId: Long) extends MapStatus with Externalizable { // loc could be null when the default constructor is called during deserialization require(loc == null || avgSize > 0 || hugeBlockSizes.size > 0 - || numNonEmptyBlocks == 0 || mapTaskId > 0, + || numNonEmptyBlocks == 0 || mapTId > 0, "Average size can only be zero for map stages that produced no output") protected def this() = this(null, -1, null, -1, null, -1) // For deserialization only @@ -189,7 +189,7 @@ private[spark] class HighlyCompressedMapStatus private ( } } - override def mapId: Long = mapTaskId + override def mapTaskId: Long = mapTId override def writeExternal(out: ObjectOutput): Unit = Utils.tryOrIOException { loc.writeExternal(out) @@ -200,7 +200,7 @@ private[spark] class HighlyCompressedMapStatus private ( out.writeInt(kv._1) out.writeByte(kv._2) } - out.writeLong(mapTaskId) + out.writeLong(mapTId) } override def readExternal(in: ObjectInput): Unit = Utils.tryOrIOException { @@ -216,7 +216,7 @@ private[spark] class HighlyCompressedMapStatus private ( hugeBlockSizesImpl(block) = size } hugeBlockSizes = hugeBlockSizesImpl - mapTaskId = in.readLong() + mapTId = in.readLong() } } @@ -224,7 +224,7 @@ private[spark] object HighlyCompressedMapStatus { def apply( loc: BlockManagerId, uncompressedSizes: Array[Long], - mapId: Long): HighlyCompressedMapStatus = { + mapTaskId: Long): HighlyCompressedMapStatus = { // We must keep track of which blocks are empty so that we don't report a zero-sized // block as being non-empty (or vice-versa) when using the average block size. var i = 0 @@ -265,6 +265,6 @@ private[spark] object HighlyCompressedMapStatus { emptyBlocks.trim() emptyBlocks.runOptimize() new HighlyCompressedMapStatus(loc, numNonEmptyBlocks, emptyBlocks, avgSize, - hugeBlockSizes, mapId) + hugeBlockSizes, mapTaskId) } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala index 135d0036ef734..a9f72eae71368 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala @@ -117,7 +117,7 @@ private[scheduler] abstract class Stage( /** Returns the sequence of partition ids that are missing (i.e. needs to be computed). */ def findMissingPartitions(): Seq[Int] - def isIndeterminate(): Boolean = { + def isIndeterminate: Boolean = { rdd.outputDeterministicLevel == DeterministicLevel.INDETERMINATE } } diff --git a/core/src/main/scala/org/apache/spark/shuffle/FetchFailedException.scala b/core/src/main/scala/org/apache/spark/shuffle/FetchFailedException.scala index 265a8acfa8d61..e822e6db90462 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/FetchFailedException.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/FetchFailedException.scala @@ -35,7 +35,7 @@ import org.apache.spark.util.Utils private[spark] class FetchFailedException( bmAddress: BlockManagerId, shuffleId: Int, - mapId: Int, + mapIndex: Int, reduceId: Int, message: String, cause: Throwable = null) @@ -44,10 +44,10 @@ private[spark] class FetchFailedException( def this( bmAddress: BlockManagerId, shuffleId: Int, - mapId: Int, + mapIndex: Int, reduceId: Int, cause: Throwable) { - this(bmAddress, shuffleId, mapId, reduceId, cause.getMessage, cause) + this(bmAddress, shuffleId, mapIndex, reduceId, cause.getMessage, cause) } // SPARK-19276. We set the fetch failure in the task context, so that even if there is user-code @@ -56,7 +56,7 @@ private[spark] class FetchFailedException( // because the TaskContext is not defined in some test cases. Option(TaskContext.get()).map(_.setFetchFailed(this)) - def toTaskFailedReason: TaskFailedReason = FetchFailed(bmAddress, shuffleId, mapId, reduceId, + def toTaskFailedReason: TaskFailedReason = FetchFailed(bmAddress, shuffleId, mapIndex, reduceId, Utils.exceptionString(this)) } diff --git a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala index 332164a7be3e7..1a4b9cdad2adb 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala @@ -51,26 +51,27 @@ private[spark] class IndexShuffleBlockResolver( private val transportConf = SparkTransportConf.fromSparkConf(conf, "shuffle") - def getDataFile(shuffleId: Int, mapId: Long): File = { - blockManager.diskBlockManager.getFile(ShuffleDataBlockId(shuffleId, mapId, NOOP_REDUCE_ID)) + def getDataFile(shuffleId: Int, mapTaskId: Long): File = { + blockManager.diskBlockManager.getFile(ShuffleDataBlockId(shuffleId, mapTaskId, NOOP_REDUCE_ID)) } - private def getIndexFile(shuffleId: Int, mapId: Long): File = { - blockManager.diskBlockManager.getFile(ShuffleIndexBlockId(shuffleId, mapId, NOOP_REDUCE_ID)) + private def getIndexFile(shuffleId: Int, mapTaskId: Long): File = { + blockManager.diskBlockManager.getFile( + ShuffleIndexBlockId(shuffleId, mapTaskId, NOOP_REDUCE_ID)) } /** * Remove data file and index file that contain the output data from one map. */ - def removeDataByMap(shuffleId: Int, mapId: Long): Unit = { - var file = getDataFile(shuffleId, mapId) + def removeDataByMap(shuffleId: Int, mapTaskId: Long): Unit = { + var file = getDataFile(shuffleId, mapTaskId) if (file.exists()) { if (!file.delete()) { logWarning(s"Error deleting data ${file.getPath()}") } } - file = getIndexFile(shuffleId, mapId) + file = getIndexFile(shuffleId, mapTaskId) if (file.exists()) { if (!file.delete()) { logWarning(s"Error deleting index ${file.getPath()}") @@ -135,13 +136,13 @@ private[spark] class IndexShuffleBlockResolver( */ def writeIndexFileAndCommit( shuffleId: Int, - mapId: Long, + mapTaskId: Long, lengths: Array[Long], dataTmp: File): Unit = { - val indexFile = getIndexFile(shuffleId, mapId) + val indexFile = getIndexFile(shuffleId, mapTaskId) val indexTmp = Utils.tempFileWith(indexFile) try { - val dataFile = getDataFile(shuffleId, mapId) + val dataFile = getDataFile(shuffleId, mapTaskId) // There is only one IndexShuffleBlockResolver per executor, this synchronization make sure // the following check and rename are atomic. synchronized { @@ -193,7 +194,7 @@ private[spark] class IndexShuffleBlockResolver( override def getBlockData(blockId: ShuffleBlockId): ManagedBuffer = { // The block is actually going to be a range of a single map output file for this map, so // find out the consolidated file, then the offset within that from our index - val indexFile = getIndexFile(blockId.shuffleId, blockId.mapId) + val indexFile = getIndexFile(blockId.shuffleId, blockId.mapTaskId) // SPARK-22982: if this FileInputStream's position is seeked forward by another piece of code // which is incorrectly using our file descriptor then this code will fetch the wrong offsets @@ -215,7 +216,7 @@ private[spark] class IndexShuffleBlockResolver( } new FileSegmentManagedBuffer( transportConf, - getDataFile(blockId.shuffleId, blockId.mapId), + getDataFile(blockId.shuffleId, blockId.mapTaskId), offset, nextOffset - offset) } finally { diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala index 7d21f32aeca6c..b397c36c72b34 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala @@ -19,13 +19,12 @@ package org.apache.spark.shuffle.sort import java.util.concurrent.ConcurrentHashMap -import scala.collection.mutable.ArrayBuffer - import org.apache.spark._ import org.apache.spark.internal.{config, Logging} import org.apache.spark.shuffle._ import org.apache.spark.shuffle.api.{ShuffleDataIO, ShuffleExecutorComponents} import org.apache.spark.util.Utils +import org.apache.spark.util.collection.OpenHashSet /** * In sort-based shuffle, incoming records are sorted according to their target partition ids, then @@ -83,7 +82,7 @@ private[spark] class SortShuffleManager(conf: SparkConf) extends ShuffleManager /** * A mapping from shuffle ids to the task ids of mappers producing output for those shuffles. */ - private[this] val taskIdMapsForShuffle = new ConcurrentHashMap[Int, ArrayBuffer[Long]]() + private[this] val taskIdMapsForShuffle = new ConcurrentHashMap[Int, OpenHashSet[Long]]() private lazy val shuffleExecutorComponents = loadShuffleExecutorComponents(conf) @@ -134,8 +133,8 @@ private[spark] class SortShuffleManager(conf: SparkConf) extends ShuffleManager context: TaskContext, metrics: ShuffleWriteMetricsReporter): ShuffleWriter[K, V] = { val mapTaskIds = taskIdMapsForShuffle.computeIfAbsent( - handle.shuffleId, _ => ArrayBuffer.empty[Long]) - mapTaskIds.synchronized { mapTaskIds.append(context.taskAttemptId()) } + handle.shuffleId, _ => new OpenHashSet[Long](16)) + mapTaskIds.synchronized { mapTaskIds.add(context.taskAttemptId()) } val env = SparkEnv.get handle match { case unsafeShuffleHandle: SerializedShuffleHandle[K @unchecked, V @unchecked] => @@ -163,9 +162,9 @@ private[spark] class SortShuffleManager(conf: SparkConf) extends ShuffleManager /** Remove a shuffle's metadata from the ShuffleManager. */ override def unregisterShuffle(shuffleId: Int): Boolean = { - Option(taskIdMapsForShuffle.remove(shuffleId)).foreach { mapIds => - mapIds.foreach { mapId => - shuffleBlockResolver.removeDataByMap(shuffleId, mapId) + Option(taskIdMapsForShuffle.remove(shuffleId)).foreach { mapTaskIds => + mapTaskIds.iterator.foreach { mapTaskId => + shuffleBlockResolver.removeDataByMap(shuffleId, mapTaskId) } } true diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala index acb09ef3b73f0..7e209f0b825fc 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala @@ -63,12 +63,12 @@ private[spark] class SortShuffleWriter[K, V, C]( // Don't bother including the time to open the merged output file in the shuffle write time, // because it just opens a single file, so is typically too fast to measure accurately // (see SPARK-3570). - val mapId = context.taskAttemptId() + val mapTaskId = context.taskAttemptId() val mapOutputWriter = shuffleExecutorComponents.createMapOutputWriter( - dep.shuffleId, mapId, dep.partitioner.numPartitions) - sorter.writePartitionedMapOutput(dep.shuffleId, mapId, mapOutputWriter) + dep.shuffleId, mapTaskId, dep.partitioner.numPartitions) + sorter.writePartitionedMapOutput(dep.shuffleId, mapTaskId, mapOutputWriter) val partitionLengths = mapOutputWriter.commitAllPartitions() - mapStatus = MapStatus(blockManager.shuffleServerId, partitionLengths, mapId) + mapStatus = MapStatus(blockManager.shuffleServerId, partitionLengths, mapTaskId) } /** Close this writer, passing along whether the map completed */ diff --git a/core/src/main/scala/org/apache/spark/storage/BlockId.scala b/core/src/main/scala/org/apache/spark/storage/BlockId.scala index 9c5b7f64e7abe..71424f89e4aaf 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockId.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockId.scala @@ -52,18 +52,18 @@ case class RDDBlockId(rddId: Int, splitIndex: Int) extends BlockId { // Format of the shuffle block ids (including data and index) should be kept in sync with // org.apache.spark.network.shuffle.ExternalShuffleBlockResolver#getBlockData(). @DeveloperApi -case class ShuffleBlockId(shuffleId: Int, mapId: Long, reduceId: Int) extends BlockId { - override def name: String = "shuffle_" + shuffleId + "_" + mapId + "_" + reduceId +case class ShuffleBlockId(shuffleId: Int, mapTaskId: Long, reduceId: Int) extends BlockId { + override def name: String = "shuffle_" + shuffleId + "_" + mapTaskId + "_" + reduceId } @DeveloperApi -case class ShuffleDataBlockId(shuffleId: Int, mapId: Long, reduceId: Int) extends BlockId { - override def name: String = "shuffle_" + shuffleId + "_" + mapId + "_" + reduceId + ".data" +case class ShuffleDataBlockId(shuffleId: Int, mapTaskId: Long, reduceId: Int) extends BlockId { + override def name: String = "shuffle_" + shuffleId + "_" + mapTaskId + "_" + reduceId + ".data" } @DeveloperApi -case class ShuffleIndexBlockId(shuffleId: Int, mapId: Long, reduceId: Int) extends BlockId { - override def name: String = "shuffle_" + shuffleId + "_" + mapId + "_" + reduceId + ".index" +case class ShuffleIndexBlockId(shuffleId: Int, mapTaskId: Long, reduceId: Int) extends BlockId { + override def name: String = "shuffle_" + shuffleId + "_" + mapTaskId + "_" + reduceId + ".index" } @DeveloperApi @@ -116,12 +116,12 @@ object BlockId { def apply(name: String): BlockId = name match { case RDD(rddId, splitIndex) => RDDBlockId(rddId.toInt, splitIndex.toInt) - case SHUFFLE(shuffleId, mapId, reduceId) => - ShuffleBlockId(shuffleId.toInt, mapId.toLong, reduceId.toInt) - case SHUFFLE_DATA(shuffleId, mapId, reduceId) => - ShuffleDataBlockId(shuffleId.toInt, mapId.toLong, reduceId.toInt) - case SHUFFLE_INDEX(shuffleId, mapId, reduceId) => - ShuffleIndexBlockId(shuffleId.toInt, mapId.toLong, reduceId.toInt) + case SHUFFLE(shuffleId, mapTaskId, reduceId) => + ShuffleBlockId(shuffleId.toInt, mapTaskId.toLong, reduceId.toInt) + case SHUFFLE_DATA(shuffleId, mapTaskId, reduceId) => + ShuffleDataBlockId(shuffleId.toInt, mapTaskId.toLong, reduceId.toInt) + case SHUFFLE_INDEX(shuffleId, mapTaskId, reduceId) => + ShuffleIndexBlockId(shuffleId.toInt, mapTaskId.toLong, reduceId.toInt) case BROADCAST(broadcastId, field) => BroadcastBlockId(broadcastId.toLong, field.stripPrefix("_")) case TASKRESULT(taskId) => diff --git a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala index 6b2bdc9e6a67e..a3c28ddf91580 100644 --- a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala @@ -227,7 +227,7 @@ final class ShuffleBlockFetcherIterator( // so we can look up the block info of each blockID val infoMap = req.blocks.map { - case (blockId, size, mapId) => (blockId.toString, (size, mapId)) + case (blockId, size, mapIndex) => (blockId.toString, (size, mapIndex)) }.toMap val remainingBlocks = new HashSet[String]() ++= infoMap.keys val blockIds = req.blocks.map(_._1.toString) @@ -300,14 +300,14 @@ final class ShuffleBlockFetcherIterator( var curRequestSize = 0L var curBlocks = new ArrayBuffer[(BlockId, Long, Int)] while (iterator.hasNext) { - val (blockId, size, mapId) = iterator.next() + val (blockId, size, mapIndex) = iterator.next() remoteBlockBytes += size if (size < 0) { throw new BlockException(blockId, "Negative block size " + size) } else if (size == 0) { throw new BlockException(blockId, "Zero-sized blocks should be excluded.") } else { - curBlocks += ((blockId, size, mapId)) + curBlocks += ((blockId, size, mapIndex)) remoteBlocks += blockId numBlocksToFetch += 1 curRequestSize += size @@ -592,7 +592,7 @@ final class ShuffleBlockFetcherIterator( e: Throwable) = { blockId match { case ShuffleBlockId(shufId, _, reduceId) => - throw new FetchFailedException(address, shufId.toInt, mapIndex.toInt, reduceId, e) + throw new FetchFailedException(address, shufId, mapIndex, reduceId, e) case _ => throw new SparkException( "Failed to get block " + blockId + ", which is not a shuffle block", e) @@ -701,7 +701,7 @@ object ShuffleBlockFetcherIterator { * @param address remote BlockManager to fetch from. * @param blocks Sequence of tuple, where the first element is the block id, * and the second element is the estimated size, used to calculate bytesInFlight, - * the third element is the mapId. + * the third element is the mapIndex. */ case class FetchRequest(address: BlockManagerId, blocks: Seq[(BlockId, Long, Int)]) { val size = blocks.map(_._2).sum diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index 73ef80980e73f..63d27076dfb2e 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -419,7 +419,7 @@ private[spark] object JsonProtocol { map(blockManagerIdToJson).getOrElse(JNothing) ("Block Manager Address" -> blockManagerAddress) ~ ("Shuffle ID" -> fetchFailed.shuffleId) ~ - ("Map ID" -> fetchFailed.mapId) ~ + ("Map ID" -> fetchFailed.mapIndex) ~ ("Reduce ID" -> fetchFailed.reduceId) ~ ("Message" -> fetchFailed.message) case exceptionFailure: ExceptionFailure => diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala index 6fecfbaca8416..45beb42f4be30 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala @@ -727,7 +727,7 @@ private[spark] class ExternalSorter[K, V, C]( */ def writePartitionedMapOutput( shuffleId: Int, - mapId: Long, + mapTaskId: Long, mapOutputWriter: ShuffleMapOutputWriter): Unit = { var nextPartitionId = 0 if (spills.isEmpty) { @@ -740,7 +740,7 @@ private[spark] class ExternalSorter[K, V, C]( var partitionPairsWriter: ShufflePartitionPairsWriter = null TryUtils.tryWithSafeFinally { partitionWriter = mapOutputWriter.getPartitionWriter(partitionId) - val blockId = ShuffleBlockId(shuffleId, mapId, partitionId) + val blockId = ShuffleBlockId(shuffleId, mapTaskId, partitionId) partitionPairsWriter = new ShufflePartitionPairsWriter( partitionWriter, serializerManager, @@ -760,7 +760,7 @@ private[spark] class ExternalSorter[K, V, C]( } else { // We must perform merge-sort; get an iterator by partition and write everything directly. for ((id, elements) <- this.partitionedIterator) { - val blockId = ShuffleBlockId(shuffleId, mapId, id) + val blockId = ShuffleBlockId(shuffleId, mapTaskId, id) var partitionWriter: ShufflePartitionWriter = null var partitionPairsWriter: ShufflePartitionPairsWriter = null TryUtils.tryWithSafeFinally { diff --git a/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala b/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala index ac7e4b51ebc2b..65523949bc5c3 100644 --- a/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala +++ b/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala @@ -528,7 +528,7 @@ class FetchFailureThrowingRDD(sc: SparkContext) extends RDD[Int](sc, Nil) { throw new FetchFailedException( bmAddress = BlockManagerId("1", "hostA", 1234), shuffleId = 0, - mapId = 0, + mapIndex = 0, reduceId = 0, message = "fake fetch failure" ) diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index ecd960ed7fbbe..3a8fd15372755 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -506,14 +506,14 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi assert(initialMapStatus1.count(_ != null) === 3) assert(initialMapStatus1.map{_.location.executorId}.toSet === Set("exec-hostA1", "exec-hostA2", "exec-hostB")) - assert(initialMapStatus1.map{_.mapId}.toSet === Set(5, 6, 7)) + assert(initialMapStatus1.map{_.mapTaskId}.toSet === Set(5, 6, 7)) val initialMapStatus2 = mapOutputTracker.shuffleStatuses(secondShuffleId).mapStatuses // val initialMapStatus1 = mapOutputTracker.mapStatuses.get(0).get assert(initialMapStatus2.count(_ != null) === 3) assert(initialMapStatus2.map{_.location.executorId}.toSet === Set("exec-hostA1", "exec-hostA2", "exec-hostB")) - assert(initialMapStatus2.map{_.mapId}.toSet === Set(8, 9, 10)) + assert(initialMapStatus2.map{_.mapTaskId}.toSet === Set(8, 9, 10)) // reduce stage fails with a fetch failure from one host complete(taskSets(2), Seq( diff --git a/core/src/test/scala/org/apache/spark/scheduler/MapStatusSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/MapStatusSuite.scala index 23cc416f8572f..700d9ebd76c0c 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/MapStatusSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/MapStatusSuite.scala @@ -92,7 +92,7 @@ class MapStatusSuite extends SparkFunSuite { val status1 = compressAndDecompressMapStatus(status) assert(status1.isInstanceOf[HighlyCompressedMapStatus]) assert(status1.location == loc) - assert(status1.mapId == mapTaskAttemptId) + assert(status1.mapTaskId == mapTaskAttemptId) for (i <- 0 until 3000) { val estimate = status1.getSizeForBlock(i) if (sizes(i) > 0) { diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala index ff435a807801e..43d7d12a3caed 100644 --- a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala @@ -350,11 +350,11 @@ class KryoSerializerSuite extends SparkFunSuite with SharedSparkContext { val ser = new KryoSerializer(conf).newInstance() val denseBlockSizes = new Array[Long](5000) val sparseBlockSizes = Array[Long](0L, 1L, 0L, 2L) - var mapTaskAttemptId = 0 + var mapTaskId = 0 Seq(denseBlockSizes, sparseBlockSizes).foreach { blockSizes => - mapTaskAttemptId += 1 + mapTaskId += 1 ser.serialize(HighlyCompressedMapStatus( - BlockManagerId("exec-1", "host", 1234), blockSizes, mapTaskAttemptId)) + BlockManagerId("exec-1", "host", 1234), blockSizes, mapTaskId)) } } diff --git a/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderSuite.scala index 76fb5d224e542..0cc1c9274fe61 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderSuite.scala @@ -87,14 +87,14 @@ class BlockStoreShuffleReaderSuite extends SparkFunSuite with LocalSparkContext // Setup the mocked BlockManager to return RecordingManagedBuffers. val localBlockManagerId = BlockManagerId("test-client", "test-client", 1) when(blockManager.blockManagerId).thenReturn(localBlockManagerId) - val buffers = (0 until numMaps).map { mapId => + val buffers = (0 until numMaps).map { mapTaskId => // Create a ManagedBuffer with the shuffle data. val nioBuffer = new NioManagedBuffer(ByteBuffer.wrap(byteOutputStream.toByteArray)) val managedBuffer = new RecordingManagedBuffer(nioBuffer) // Setup the blockManager mock so the buffer gets returned when the shuffle code tries to // fetch shuffle data. - val shuffleBlockId = ShuffleBlockId(shuffleId, mapId, reduceId) + val shuffleBlockId = ShuffleBlockId(shuffleId, mapTaskId, reduceId) when(blockManager.getBlockData(shuffleBlockId)).thenReturn(managedBuffer) managedBuffer } @@ -106,9 +106,9 @@ class BlockStoreShuffleReaderSuite extends SparkFunSuite with LocalSparkContext shuffleId, reduceId, reduceId + 1, false)).thenReturn { // Test a scenario where all data is local, to avoid creating a bunch of additional mocks // for the code to read data over the network. - val shuffleBlockIdsAndSizes = (0 until numMaps).map { mapId => - val shuffleBlockId = ShuffleBlockId(shuffleId, mapId, reduceId) - (shuffleBlockId, byteOutputStream.size().toLong, mapId) + val shuffleBlockIdsAndSizes = (0 until numMaps).map { mapTaskId => + val shuffleBlockId = ShuffleBlockId(shuffleId, mapTaskId, reduceId) + (shuffleBlockId, byteOutputStream.size().toLong, mapTaskId) } Seq((localBlockManagerId, shuffleBlockIdsAndSizes)).toIterator } diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/IndexShuffleBlockResolverSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/IndexShuffleBlockResolverSuite.scala index 27bb06b4e0636..b087c2fbd0c88 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/IndexShuffleBlockResolverSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/IndexShuffleBlockResolverSuite.scala @@ -60,8 +60,8 @@ class IndexShuffleBlockResolverSuite extends SparkFunSuite with BeforeAndAfterEa test("commit shuffle files multiple times") { val shuffleId = 1 - val mapId = 2 - val idxName = s"shuffle_${shuffleId}_${mapId}_0.index" + val mapTaskId = 2 + val idxName = s"shuffle_${shuffleId}_${mapTaskId}_0.index" val resolver = new IndexShuffleBlockResolver(conf, blockManager) val lengths = Array[Long](10, 0, 20) val dataTmp = File.createTempFile("shuffle", null, tempDir) @@ -71,10 +71,10 @@ class IndexShuffleBlockResolverSuite extends SparkFunSuite with BeforeAndAfterEa } { out.close() } - resolver.writeIndexFileAndCommit(shuffleId, mapId, lengths, dataTmp) + resolver.writeIndexFileAndCommit(shuffleId, mapTaskId, lengths, dataTmp) val indexFile = new File(tempDir.getAbsolutePath, idxName) - val dataFile = resolver.getDataFile(shuffleId, mapId) + val dataFile = resolver.getDataFile(shuffleId, mapTaskId) assert(indexFile.exists()) assert(indexFile.length() === (lengths.length + 1) * 8) @@ -91,7 +91,7 @@ class IndexShuffleBlockResolverSuite extends SparkFunSuite with BeforeAndAfterEa } { out2.close() } - resolver.writeIndexFileAndCommit(shuffleId, mapId, lengths2, dataTmp2) + resolver.writeIndexFileAndCommit(shuffleId, mapTaskId, lengths2, dataTmp2) assert(indexFile.length() === (lengths.length + 1) * 8) assert(lengths2.toSeq === lengths.toSeq) @@ -130,7 +130,7 @@ class IndexShuffleBlockResolverSuite extends SparkFunSuite with BeforeAndAfterEa } { out3.close() } - resolver.writeIndexFileAndCommit(shuffleId, mapId, lengths3, dataTmp3) + resolver.writeIndexFileAndCommit(shuffleId, mapTaskId, lengths3, dataTmp3) assert(indexFile.length() === (lengths3.length + 1) * 8) assert(lengths3.toSeq != lengths.toSeq) assert(dataFile.exists()) diff --git a/core/src/test/scala/org/apache/spark/storage/BlockIdSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockIdSuite.scala index ff4755833a916..8d6de10daa5b1 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockIdSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockIdSuite.scala @@ -58,7 +58,7 @@ class BlockIdSuite extends SparkFunSuite { assert(id.name === "shuffle_1_2_3") assert(id.asRDDId === None) assert(id.shuffleId === 1) - assert(id.mapId === 2) + assert(id.mapTaskId === 2) assert(id.reduceId === 3) assert(id.isShuffle) assertSame(id, BlockId(id.toString)) @@ -71,7 +71,7 @@ class BlockIdSuite extends SparkFunSuite { assert(id.name === "shuffle_4_5_6.data") assert(id.asRDDId === None) assert(id.shuffleId === 4) - assert(id.mapId === 5) + assert(id.mapTaskId === 5) assert(id.reduceId === 6) assert(!id.isShuffle) assertSame(id, BlockId(id.toString)) @@ -84,7 +84,7 @@ class BlockIdSuite extends SparkFunSuite { assert(id.name === "shuffle_7_8_9.index") assert(id.asRDDId === None) assert(id.shuffleId === 7) - assert(id.mapId === 8) + assert(id.mapTaskId === 8) assert(id.reduceId === 9) assert(!id.isShuffle) assertSame(id, BlockId(id.toString)) diff --git a/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala b/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala index 05c21492b93e9..e5a615c2c2cbb 100644 --- a/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala @@ -99,8 +99,8 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT val transfer = createMockTransfer(remoteBlocks) val blocksByAddress = Seq[(BlockManagerId, Seq[(BlockId, Long, Int)])]( - (localBmId, localBlocks.keys.map(blockId => (blockId, 1.asInstanceOf[Long], 0)).toSeq), - (remoteBmId, remoteBlocks.keys.map(blockId => (blockId, 1.asInstanceOf[Long], 1)).toSeq) + (localBmId, localBlocks.keys.map(blockId => (blockId, 1L, 0)).toSeq), + (remoteBmId, remoteBlocks.keys.map(blockId => (blockId, 1L, 1)).toSeq) ).toIterator val taskContext = TaskContext.empty() @@ -180,7 +180,7 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT }) val blocksByAddress = Seq[(BlockManagerId, Seq[(BlockId, Long, Int)])]( - (remoteBmId, blocks.keys.map(blockId => (blockId, 1.asInstanceOf[Long], 0)).toSeq)).toIterator + (remoteBmId, blocks.keys.map(blockId => (blockId, 1L, 0)).toSeq)).toIterator val taskContext = TaskContext.empty() val iterator = new ShuffleBlockFetcherIterator( @@ -248,7 +248,7 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT }) val blocksByAddress = Seq[(BlockManagerId, Seq[(BlockId, Long, Int)])]( - (remoteBmId, blocks.keys.map(blockId => (blockId, 1.asInstanceOf[Long], 0)).toSeq)) + (remoteBmId, blocks.keys.map(blockId => (blockId, 1L, 0)).toSeq)) .toIterator val taskContext = TaskContext.empty() @@ -338,7 +338,7 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT }) val blocksByAddress = Seq[(BlockManagerId, Seq[(BlockId, Long, Int)])]( - (remoteBmId, blocks.keys.map(blockId => (blockId, 1.asInstanceOf[Long], 0)).toSeq)).toIterator + (remoteBmId, blocks.keys.map(blockId => (blockId, 1L, 0)).toSeq)).toIterator val taskContext = TaskContext.empty() val iterator = new ShuffleBlockFetcherIterator( @@ -533,7 +533,7 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT }) val blocksByAddress = Seq[(BlockManagerId, Seq[(BlockId, Long, Int)])]( - (remoteBmId, blocks.keys.map(blockId => (blockId, 1.asInstanceOf[Long], 0)).toSeq)) + (remoteBmId, blocks.keys.map(blockId => (blockId, 1L, 0)).toSeq)) .toIterator val taskContext = TaskContext.empty() @@ -643,7 +643,7 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT val transfer = createMockTransfer(blocks.mapValues(_ => createMockManagedBuffer(0))) val blocksByAddress = Seq[(BlockManagerId, Seq[(BlockId, Long, Int)])]( - (remoteBmId, blocks.keys.map(blockId => (blockId, 1.asInstanceOf[Long], 0)).toSeq)) + (remoteBmId, blocks.keys.map(blockId => (blockId, 1L, 0)).toSeq)) val taskContext = TaskContext.empty() val iterator = new ShuffleBlockFetcherIterator( diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index e781c5f71faf4..84d7084209ae9 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -731,7 +731,7 @@ private[spark] object JsonProtocolSuite extends Assertions { case (Resubmitted, Resubmitted) => case (r1: FetchFailed, r2: FetchFailed) => assert(r1.shuffleId === r2.shuffleId) - assert(r1.mapId === r2.mapId) + assert(r1.mapIndex === r2.mapIndex) assert(r1.reduceId === r2.reduceId) assert(r1.bmAddress === r2.bmAddress) assert(r1.message === r2.message) diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 09499e04ac7f7..c2b9c96e4a05b 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -421,21 +421,22 @@ object MimaExcludes { // [SPARK-25341][CORE] Support rolling back a shuffle map stage and re-generate the shuffle files ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.shuffle.sort.UnsafeShuffleWriter.this"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.storage.ShuffleIndexBlockId.mapId"), ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.storage.ShuffleIndexBlockId.copy$default$2"), ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.storage.ShuffleIndexBlockId.copy"), ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.storage.ShuffleIndexBlockId.this"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.storage.ShuffleDataBlockId.mapId"), ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.storage.ShuffleDataBlockId.copy$default$2"), ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.storage.ShuffleDataBlockId.copy"), ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.storage.ShuffleDataBlockId.this"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.storage.ShuffleBlockId.mapId"), ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.storage.ShuffleBlockId.copy$default$2"), ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.storage.ShuffleBlockId.copy"), ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.storage.ShuffleBlockId.this"), ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.storage.ShuffleIndexBlockId.apply"), ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.storage.ShuffleDataBlockId.apply"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.storage.ShuffleBlockId.apply") + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.storage.ShuffleBlockId.apply"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.storage.ShuffleIndexBlockId.mapId"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.storage.ShuffleDataBlockId.mapId"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.storage.ShuffleBlockId.mapId"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.FetchFailed.mapId") ) // Exclude rules for 2.4.x From 2bb43882cb27501641b5b24c0ee0125ff8018a0a Mon Sep 17 00:00:00 2001 From: Yuanjian Li Date: Fri, 6 Sep 2019 16:06:32 +0800 Subject: [PATCH 07/15] fix nit comment --- core/src/main/scala/org/apache/spark/MapOutputTracker.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 44b247c10fd63..059c1b368570f 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -292,7 +292,7 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging * endPartition is excluded from the range). * * @return A sequence of 2-item tuples, where the first item in the tuple is a BlockManagerId, - * and the second item is a sequence of (shuffle block id, shuffle block size, map id) + * and the second item is a sequence of (shuffle block id, shuffle block size, map index) * tuples describing the shuffle blocks that are stored at that block manager. */ def getMapSizesByExecutorId( @@ -846,7 +846,7 @@ private[spark] object MapOutputTracker extends Logging { * @param shuffleId Identifier for the shuffle * @param startPartition Start of map output partition ID range (included in range) * @param endPartition End of map output partition ID range (excluded from range) - * @param statuses List of map statuses, indexed by map ID. + * @param statuses List of map statuses, indexed by map partition index. * @param useOldFetchProtocol Whether to use the old shuffle fetch protocol. * @return A sequence of 2-item tuples, where the first item in the tuple is a BlockManagerId, * and the second item is a sequence of (shuffle block id, shuffle block size, map index) From 061e363731dae004aea3f7551e8fc842537cd9c5 Mon Sep 17 00:00:00 2001 From: Yuanjian Li Date: Sun, 8 Sep 2019 10:45:06 +0800 Subject: [PATCH 08/15] Add mapTaskId in FetchFailedException, address some comments --- .../network/shuffle/ExternalBlockHandler.java | 1 + .../org/apache/spark/TaskEndReason.scala | 3 +- .../apache/spark/scheduler/DAGScheduler.scala | 2 +- .../apache/spark/scheduler/MapStatus.scala | 26 ++++----- .../spark/shuffle/FetchFailedException.scala | 10 ++-- .../storage/ShuffleBlockFetcherIterator.scala | 4 +- .../org/apache/spark/util/JsonProtocol.scala | 10 ++-- .../spark/InternalAccumulatorSuite.scala | 1 + .../org/apache/spark/SparkContextSuite.scala | 2 +- .../apache/spark/executor/ExecutorSuite.scala | 1 + .../spark/scheduler/DAGSchedulerSuite.scala | 58 ++++++++++--------- .../OutputCommitCoordinatorSuite.scala | 2 +- .../scheduler/SchedulerIntegrationSuite.scala | 2 +- .../spark/scheduler/TaskContextSuite.scala | 2 +- .../spark/scheduler/TaskSetManagerSuite.scala | 4 +- .../org/apache/spark/ui/UISeleniumSuite.scala | 6 +- .../apache/spark/util/JsonProtocolSuite.scala | 9 +-- 17 files changed, 79 insertions(+), 64 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 e0a1e263e419e..1d8303c1aa3f5 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 @@ -337,6 +337,7 @@ public boolean hasNext() { // mapTaskIds.length must equal to reduceIds.length, and the passed in FetchShuffleBlocks // must have non-empty mapTaskIds and reduceIds, see the checking logic in // OneForOneBlockFetcher. + assert(mapTaskIds.length != 0 && mapTaskIds.length == reduceIds.length); return mapIdx < mapTaskIds.length && reduceIdx < reduceIds[mapIdx].length; } diff --git a/core/src/main/scala/org/apache/spark/TaskEndReason.scala b/core/src/main/scala/org/apache/spark/TaskEndReason.scala index 9670d72913f21..199d7f41349e6 100644 --- a/core/src/main/scala/org/apache/spark/TaskEndReason.scala +++ b/core/src/main/scala/org/apache/spark/TaskEndReason.scala @@ -83,6 +83,7 @@ case object Resubmitted extends TaskFailedReason { case class FetchFailed( bmAddress: BlockManagerId, // Note that bmAddress can be null shuffleId: Int, + mapTaskId: Long, mapIndex: Int, reduceId: Int, message: String) @@ -90,7 +91,7 @@ case class FetchFailed( override def toErrorString: String = { val bmAddressString = if (bmAddress == null) "null" else bmAddress.toString s"FetchFailed($bmAddressString, shuffleId=$shuffleId, mapIndex=$mapIndex, " + - s"reduceId=$reduceId, message=\n$message\n)" + s"mapTaskId=$mapTaskId, reduceId=$reduceId, message=\n$message\n)" } /** diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 634b2f802c908..ca6e1847ce704 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1507,7 +1507,7 @@ private[spark] class DAGScheduler( } } - case FetchFailed(bmAddress, shuffleId, mapIndex, _, failureMessage) => + case FetchFailed(bmAddress, shuffleId, _, mapIndex, _, failureMessage) => val failedStage = stageIdToStage(task.stageId) val mapStage = shuffleIdToMapStage(shuffleId) diff --git a/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala b/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala index 75dc8debdd1a6..c9d37c985d211 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala @@ -108,19 +108,19 @@ private[spark] object MapStatus { * * @param loc location where the task is being executed. * @param compressedSizes size of the blocks, indexed by reduce partition id. - * @param mapTId unique task id for the task + * @param _mapTaskId unique task id for the task */ private[spark] class CompressedMapStatus( private[this] var loc: BlockManagerId, private[this] var compressedSizes: Array[Byte], - private[this] var mapTId: Long) + private[this] var _mapTaskId: Long) extends MapStatus with Externalizable { // For deserialization only protected def this() = this(null, null.asInstanceOf[Array[Byte]], -1) - def this(loc: BlockManagerId, uncompressedSizes: Array[Long], taskAttemptId: Long) { - this(loc, uncompressedSizes.map(MapStatus.compressSize), taskAttemptId) + def this(loc: BlockManagerId, uncompressedSizes: Array[Long], mapTaskId: Long) { + this(loc, uncompressedSizes.map(MapStatus.compressSize), mapTaskId) } override def location: BlockManagerId = loc @@ -129,13 +129,13 @@ private[spark] class CompressedMapStatus( MapStatus.decompressSize(compressedSizes(reduceId)) } - override def mapTaskId: Long = mapTId + override def mapTaskId: Long = _mapTaskId override def writeExternal(out: ObjectOutput): Unit = Utils.tryOrIOException { loc.writeExternal(out) out.writeInt(compressedSizes.length) out.write(compressedSizes) - out.writeLong(mapTId) + out.writeLong(_mapTaskId) } override def readExternal(in: ObjectInput): Unit = Utils.tryOrIOException { @@ -143,7 +143,7 @@ private[spark] class CompressedMapStatus( val len = in.readInt() compressedSizes = new Array[Byte](len) in.readFully(compressedSizes) - mapTId = in.readLong() + _mapTaskId = in.readLong() } } @@ -157,7 +157,7 @@ private[spark] class CompressedMapStatus( * @param emptyBlocks a bitmap tracking which blocks are empty * @param avgSize average size of the non-empty and non-huge blocks * @param hugeBlockSizes sizes of huge blocks by their reduceId. - * @param mapTId unique task id for the task + * @param _mapTaskId unique task id for the task */ private[spark] class HighlyCompressedMapStatus private ( private[this] var loc: BlockManagerId, @@ -165,12 +165,12 @@ private[spark] class HighlyCompressedMapStatus private ( private[this] var emptyBlocks: RoaringBitmap, private[this] var avgSize: Long, private[this] var hugeBlockSizes: scala.collection.Map[Int, Byte], - private[this] var mapTId: Long) + private[this] var _mapTaskId: Long) extends MapStatus with Externalizable { // loc could be null when the default constructor is called during deserialization require(loc == null || avgSize > 0 || hugeBlockSizes.size > 0 - || numNonEmptyBlocks == 0 || mapTId > 0, + || numNonEmptyBlocks == 0 || _mapTaskId > 0, "Average size can only be zero for map stages that produced no output") protected def this() = this(null, -1, null, -1, null, -1) // For deserialization only @@ -189,7 +189,7 @@ private[spark] class HighlyCompressedMapStatus private ( } } - override def mapTaskId: Long = mapTId + override def mapTaskId: Long = _mapTaskId override def writeExternal(out: ObjectOutput): Unit = Utils.tryOrIOException { loc.writeExternal(out) @@ -200,7 +200,7 @@ private[spark] class HighlyCompressedMapStatus private ( out.writeInt(kv._1) out.writeByte(kv._2) } - out.writeLong(mapTId) + out.writeLong(_mapTaskId) } override def readExternal(in: ObjectInput): Unit = Utils.tryOrIOException { @@ -216,7 +216,7 @@ private[spark] class HighlyCompressedMapStatus private ( hugeBlockSizesImpl(block) = size } hugeBlockSizes = hugeBlockSizesImpl - mapTId = in.readLong() + _mapTaskId = in.readLong() } } diff --git a/core/src/main/scala/org/apache/spark/shuffle/FetchFailedException.scala b/core/src/main/scala/org/apache/spark/shuffle/FetchFailedException.scala index e822e6db90462..965546ca9fb5f 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/FetchFailedException.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/FetchFailedException.scala @@ -35,6 +35,7 @@ import org.apache.spark.util.Utils private[spark] class FetchFailedException( bmAddress: BlockManagerId, shuffleId: Int, + mapTaskId: Long, mapIndex: Int, reduceId: Int, message: String, @@ -44,10 +45,11 @@ private[spark] class FetchFailedException( def this( bmAddress: BlockManagerId, shuffleId: Int, + mapTaskId: Long, mapIndex: Int, reduceId: Int, cause: Throwable) { - this(bmAddress, shuffleId, mapIndex, reduceId, cause.getMessage, cause) + this(bmAddress, shuffleId, mapTaskId, mapIndex, reduceId, cause.getMessage, cause) } // SPARK-19276. We set the fetch failure in the task context, so that even if there is user-code @@ -56,8 +58,8 @@ private[spark] class FetchFailedException( // because the TaskContext is not defined in some test cases. Option(TaskContext.get()).map(_.setFetchFailed(this)) - def toTaskFailedReason: TaskFailedReason = FetchFailed(bmAddress, shuffleId, mapIndex, reduceId, - Utils.exceptionString(this)) + def toTaskFailedReason: TaskFailedReason = FetchFailed( + bmAddress, shuffleId, mapTaskId, mapIndex, reduceId, Utils.exceptionString(this)) } /** @@ -67,4 +69,4 @@ private[spark] class MetadataFetchFailedException( shuffleId: Int, reduceId: Int, message: String) - extends FetchFailedException(null, shuffleId, -1, reduceId, message) + extends FetchFailedException(null, shuffleId, -1L, -1, reduceId, message) diff --git a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala index a3c28ddf91580..d0869519a2f74 100644 --- a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala @@ -591,8 +591,8 @@ final class ShuffleBlockFetcherIterator( address: BlockManagerId, e: Throwable) = { blockId match { - case ShuffleBlockId(shufId, _, reduceId) => - throw new FetchFailedException(address, shufId, mapIndex, reduceId, e) + case ShuffleBlockId(shufId, mapTaskId, reduceId) => + throw new FetchFailedException(address, shufId, mapTaskId, mapIndex, reduceId, e) case _ => throw new SparkException( "Failed to get block " + blockId + ", which is not a shuffle block", e) diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index 63d27076dfb2e..c3e501d61ab52 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -419,7 +419,8 @@ private[spark] object JsonProtocol { map(blockManagerIdToJson).getOrElse(JNothing) ("Block Manager Address" -> blockManagerAddress) ~ ("Shuffle ID" -> fetchFailed.shuffleId) ~ - ("Map ID" -> fetchFailed.mapIndex) ~ + ("Map Task ID" -> fetchFailed.mapTaskId) ~ + ("Map Index" -> fetchFailed.mapIndex) ~ ("Reduce ID" -> fetchFailed.reduceId) ~ ("Message" -> fetchFailed.message) case exceptionFailure: ExceptionFailure => @@ -974,11 +975,12 @@ private[spark] object JsonProtocol { case `fetchFailed` => val blockManagerAddress = blockManagerIdFromJson(json \ "Block Manager Address") val shuffleId = (json \ "Shuffle ID").extract[Int] - val mapId = (json \ "Map ID").extract[Int] + val mapTaskId = (json \ "Map Task ID").extract[Long] + val mapIndex = (json \ "Map Index").extract[Int] val reduceId = (json \ "Reduce ID").extract[Int] val message = jsonOption(json \ "Message").map(_.extract[String]) - new FetchFailed(blockManagerAddress, shuffleId, mapId, reduceId, - message.getOrElse("Unknown reason")) + new FetchFailed(blockManagerAddress, shuffleId, mapTaskId, mapIndex, + reduceId, message.getOrElse("Unknown reason")) case `exceptionFailure` => val className = (json \ "Class Name").extract[String] val description = (json \ "Description").extract[String] diff --git a/core/src/test/scala/org/apache/spark/InternalAccumulatorSuite.scala b/core/src/test/scala/org/apache/spark/InternalAccumulatorSuite.scala index e7eef8ec5150c..8433a6f52ac7a 100644 --- a/core/src/test/scala/org/apache/spark/InternalAccumulatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/InternalAccumulatorSuite.scala @@ -142,6 +142,7 @@ class InternalAccumulatorSuite extends SparkFunSuite with LocalSparkContext { sid, taskContext.partitionId(), taskContext.partitionId(), + taskContext.partitionId(), "simulated fetch failure") } else { iter diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala index 786f55c96a3e8..ac54e5ef10fe9 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala @@ -705,7 +705,7 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu if (context.stageAttemptNumber == 0) { if (context.partitionId == 0) { // Make the first task in the first stage attempt fail. - throw new FetchFailedException(SparkEnv.get.blockManager.blockManagerId, 0, 0, 0, + throw new FetchFailedException(SparkEnv.get.blockManager.blockManagerId, 0, 0L, 0, 0, new java.io.IOException("fake")) } else { // Make the second task in the first stage attempt sleep to generate a zombie task diff --git a/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala b/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala index 65523949bc5c3..b5f563b2aeeea 100644 --- a/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala +++ b/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala @@ -528,6 +528,7 @@ class FetchFailureThrowingRDD(sc: SparkContext) extends RDD[Int](sc, Nil) { throw new FetchFailedException( bmAddress = BlockManagerId("1", "hostA", 1234), shuffleId = 0, + mapTaskId = 0L, mapIndex = 0, reduceId = 0, message = "fake fetch failure" diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 3a8fd15372755..5800a2c85307d 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -517,7 +517,8 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // reduce stage fails with a fetch failure from one host complete(taskSets(2), Seq( - (FetchFailed(BlockManagerId("exec-hostA2", "hostA", 12345), firstShuffleId, 0, 0, "ignored"), + (FetchFailed(BlockManagerId("exec-hostA2", "hostA", 12345), + firstShuffleId, 0L, 0, 0, "ignored"), null) )) @@ -759,7 +760,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // the 2nd ResultTask failed complete(taskSets(1), Seq( (Success, 42), - (FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, "ignored"), null))) + (FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0L, 0, 0, "ignored"), null))) // this will get called // blockManagerMaster.removeExecutor("exec-hostA") // ask the scheduler to try it again @@ -906,7 +907,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi val stageAttempt = taskSets.last checkStageId(stageId, attemptIdx, stageAttempt) complete(stageAttempt, stageAttempt.tasks.zipWithIndex.map { case (task, idx) => - (FetchFailed(makeBlockManagerId("hostA"), shuffleDep.shuffleId, 0, idx, "ignored"), null) + (FetchFailed(makeBlockManagerId("hostA"), shuffleDep.shuffleId, 0L, 0, idx, "ignored"), null) }.toSeq) } @@ -1139,14 +1140,14 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // The first result task fails, with a fetch failure for the output from the first mapper. runEvent(makeCompletionEvent( taskSets(1).tasks(0), - FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, "ignored"), + FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0L, 0, 0, "ignored"), null)) assert(sparkListener.failedStages.contains(1)) // The second ResultTask fails, with a fetch failure for the output from the second mapper. runEvent(makeCompletionEvent( taskSets(1).tasks(0), - FetchFailed(makeBlockManagerId("hostA"), shuffleId, 1, 1, "ignored"), + FetchFailed(makeBlockManagerId("hostA"), shuffleId, 1L, 1, 1, "ignored"), null)) // The SparkListener should not receive redundant failure events. assert(sparkListener.failedStages.size === 1) @@ -1166,7 +1167,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // The first result task fails, with a fetch failure for the output from the first mapper. runEvent(makeCompletionEvent( taskSets(1).tasks(0), - FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, "ignored"), + FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0L, 0, 0, "ignored"), null)) assert(mapOutputTracker.findMissingPartitions(shuffleId) === Some(Seq(0, 1))) @@ -1268,7 +1269,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // The first result task fails, with a fetch failure for the output from the first mapper. runEvent(makeCompletionEvent( taskSets(1).tasks(0), - FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, "ignored"), + FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0L, 0, 0, "ignored"), null)) assert(sparkListener.failedStages.contains(1)) @@ -1281,7 +1282,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // The second ResultTask fails, with a fetch failure for the output from the second mapper. runEvent(makeCompletionEvent( taskSets(1).tasks(1), - FetchFailed(makeBlockManagerId("hostB"), shuffleId, 1, 1, "ignored"), + FetchFailed(makeBlockManagerId("hostB"), shuffleId, 1L, 1, 1, "ignored"), null)) // Another ResubmitFailedStages event should not result in another attempt for the map @@ -1327,7 +1328,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // The first result task fails, with a fetch failure for the output from the first mapper. runEvent(makeCompletionEvent( taskSets(1).tasks(0), - FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, "ignored"), + FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0L, 0, 0, "ignored"), null)) // Trigger resubmission of the failed map stage and finish the re-started map task. @@ -1342,7 +1343,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // A late FetchFailed arrives from the second task in the original reduce stage. runEvent(makeCompletionEvent( taskSets(1).tasks(1), - FetchFailed(makeBlockManagerId("hostB"), shuffleId, 1, 1, "ignored"), + FetchFailed(makeBlockManagerId("hostB"), shuffleId, 1L, 1, 1, "ignored"), null)) // Running ResubmitFailedStages shouldn't result in any more attempts for the map stage, because @@ -1537,7 +1538,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi runEvent(ExecutorLost("exec-hostA", ExecutorKilled)) runEvent(makeCompletionEvent( taskSets(1).tasks(0), - FetchFailed(null, firstShuffleId, 2, 0, "Fetch failed"), + FetchFailed(null, firstShuffleId, 2L, 2, 0, "Fetch failed"), null)) // so we resubmit stage 0, which completes happily @@ -1796,7 +1797,8 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // lets say there is a fetch failure in this task set, which makes us go back and // run stage 0, attempt 1 complete(taskSets(1), Seq( - (FetchFailed(makeBlockManagerId("hostA"), shuffleDep1.shuffleId, 0, 0, "ignored"), null))) + (FetchFailed(makeBlockManagerId("hostA"), + shuffleDep1.shuffleId, 0L, 0, 0, "ignored"), null))) scheduler.resubmitFailedStages() // stage 0, attempt 1 should have the properties of job2 @@ -1877,7 +1879,8 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi (Success, makeMapStatus("hostC", 1)))) // fail the third stage because hostA went down complete(taskSets(2), Seq( - (FetchFailed(makeBlockManagerId("hostA"), shuffleDepTwo.shuffleId, 0, 0, "ignored"), null))) + (FetchFailed(makeBlockManagerId("hostA"), + shuffleDepTwo.shuffleId, 0L, 0, 0, "ignored"), null))) // TODO assert this: // blockManagerMaster.removeExecutor("exec-hostA") // have DAGScheduler try again @@ -1908,7 +1911,8 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi (Success, makeMapStatus("hostB", 1)))) // pretend stage 2 failed because hostA went down complete(taskSets(2), Seq( - (FetchFailed(makeBlockManagerId("hostA"), shuffleDepTwo.shuffleId, 0, 0, "ignored"), null))) + (FetchFailed(makeBlockManagerId("hostA"), + shuffleDepTwo.shuffleId, 0L, 0, 0, "ignored"), null))) // TODO assert this: // blockManagerMaster.removeExecutor("exec-hostA") // DAGScheduler should notice the cached copy of the second shuffle and try to get it rerun. @@ -2269,7 +2273,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi submit(reduceRdd, Array(0, 1)) complete(taskSets(1), Seq( (Success, 42), - (FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, "ignored"), null))) + (FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0L, 0, 0, "ignored"), null))) // Ask the scheduler to try it again; TaskSet 2 will rerun the map task that we couldn't fetch // from, then TaskSet 3 will run the reduce stage scheduler.resubmitFailedStages() @@ -2328,7 +2332,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi assert(taskSets(1).stageId === 1) complete(taskSets(1), Seq( (Success, makeMapStatus("hostA", rdd2.partitions.length)), - (FetchFailed(makeBlockManagerId("hostA"), dep1.shuffleId, 0, 0, "ignored"), null))) + (FetchFailed(makeBlockManagerId("hostA"), dep1.shuffleId, 0L, 0, 0, "ignored"), null))) scheduler.resubmitFailedStages() assert(listener2.results.size === 0) // Second stage listener should not have a result yet @@ -2354,7 +2358,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi assert(taskSets(4).stageId === 2) complete(taskSets(4), Seq( (Success, 52), - (FetchFailed(makeBlockManagerId("hostD"), dep2.shuffleId, 0, 0, "ignored"), null))) + (FetchFailed(makeBlockManagerId("hostD"), dep2.shuffleId, 0L, 0, 0, "ignored"), null))) scheduler.resubmitFailedStages() // TaskSet 5 will rerun stage 1's lost task, then TaskSet 6 will rerun stage 2 @@ -2392,7 +2396,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi assert(taskSets(1).stageId === 1) complete(taskSets(1), Seq( (Success, makeMapStatus("hostC", rdd2.partitions.length)), - (FetchFailed(makeBlockManagerId("hostA"), dep1.shuffleId, 0, 0, "ignored"), null))) + (FetchFailed(makeBlockManagerId("hostA"), dep1.shuffleId, 0L, 0, 0, "ignored"), null))) scheduler.resubmitFailedStages() // Stage1 listener should not have a result yet assert(listener2.results.size === 0) @@ -2527,7 +2531,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi rdd1.map { case (x, _) if (x == 1) => throw new FetchFailedException( - BlockManagerId("1", "1", 1), shuffleHandle.shuffleId, 0, 0, "test") + BlockManagerId("1", "1", 1), shuffleHandle.shuffleId, 0L, 0, 0, "test") case (x, _) => x }.count() } @@ -2540,7 +2544,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi rdd1.map { case (x, _) if (x == 1) && FailThisAttempt._fail.getAndSet(false) => throw new FetchFailedException( - BlockManagerId("1", "1", 1), shuffleHandle.shuffleId, 0, 0, "test") + BlockManagerId("1", "1", 1), shuffleHandle.shuffleId, 0L, 0, 0, "test") } } @@ -2594,7 +2598,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi assert(taskSets(1).stageId === 1 && taskSets(1).stageAttemptId === 0) runEvent(makeCompletionEvent( taskSets(1).tasks(0), - FetchFailed(makeBlockManagerId("hostA"), shuffleIdA, 0, 0, + FetchFailed(makeBlockManagerId("hostA"), shuffleIdA, 0L, 0, 0, "Fetch failure of task: stageId=1, stageAttempt=0, partitionId=0"), result = null)) @@ -2775,7 +2779,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // The first task of the final stage failed with fetch failure runEvent(makeCompletionEvent( taskSets(2).tasks(0), - FetchFailed(makeBlockManagerId("hostC"), shuffleId2, 0, 0, "ignored"), + FetchFailed(makeBlockManagerId("hostC"), shuffleId2, 0L, 0, 0, "ignored"), null)) (shuffleId1, shuffleId2) } @@ -2814,7 +2818,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // The first task of the `shuffleMapRdd2` failed with fetch failure runEvent(makeCompletionEvent( taskSets(3).tasks(0), - FetchFailed(makeBlockManagerId("hostA"), shuffleId1, 0, 0, "ignored"), + FetchFailed(makeBlockManagerId("hostA"), shuffleId1, 0L, 0, 0, "ignored"), null)) val newFailedStages = scheduler.failedStages.toSeq @@ -2879,7 +2883,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // Executor lost on hostB, both of stage 0 and 1 should be reran. runEvent(makeCompletionEvent( taskSets(2).tasks(0), - FetchFailed(makeBlockManagerId("hostB"), shuffleId2, 0, 0, "ignored"), + FetchFailed(makeBlockManagerId("hostB"), shuffleId2, 0L, 0, 0, "ignored"), null)) mapOutputTracker.removeOutputsOnHost("hostB") @@ -2941,7 +2945,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // Fail the second task with FetchFailed. runEvent(makeCompletionEvent( taskSets.last.tasks(1), - FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, "ignored"), + FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0L, 0, 0, "ignored"), null)) // The job should fail because Spark can't rollback the result stage. @@ -2984,7 +2988,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // Fail the second task with FetchFailed. runEvent(makeCompletionEvent( taskSets.last.tasks(1), - FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, "ignored"), + FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0L, 0, 0, "ignored"), null)) assert(failure == null, "job should not fail") @@ -3047,7 +3051,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // The second task of the shuffle map stage failed with FetchFailed. runEvent(makeCompletionEvent( taskSets(0).tasks(1), - FetchFailed(makeBlockManagerId("hostB"), shuffleDep.shuffleId, 0, 0, "ignored"), + FetchFailed(makeBlockManagerId("hostB"), shuffleDep.shuffleId, 0L, 0, 0, "ignored"), null)) scheduler.resubmitFailedStages() diff --git a/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala index d6964063c118e..6f80c7c0fe817 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala @@ -254,7 +254,7 @@ class OutputCommitCoordinatorSuite extends SparkFunSuite with BeforeAndAfter { .reduceByKey { case (_, _) => val ctx = TaskContext.get() if (ctx.stageAttemptNumber() == 0) { - throw new FetchFailedException(SparkEnv.get.blockManager.blockManagerId, 1, 1, 1, + throw new FetchFailedException(SparkEnv.get.blockManager.blockManagerId, 1, 1L, 1, 1, new Exception("Failure for test.")) } else { ctx.stageId() diff --git a/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala index 96706536fe53c..4f737c9499ad6 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala @@ -621,7 +621,7 @@ class BasicSchedulerIntegrationSuite extends SchedulerIntegrationSuite[SingleCor backend.taskSuccess(taskDescription, DAGSchedulerSuite.makeMapStatus("hostA", 10)) case (1, 0, 0) => val fetchFailed = FetchFailed( - DAGSchedulerSuite.makeBlockManagerId("hostA"), shuffleId, 0, 0, "ignored") + DAGSchedulerSuite.makeBlockManagerId("hostA"), shuffleId, 0L, 0, 0, "ignored") backend.taskFailed(taskDescription, fetchFailed) case (1, _, partition) => backend.taskSuccess(taskDescription, 42 + partition) diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala index c16b552d20891..394a2a9fbf7cb 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala @@ -176,7 +176,7 @@ class TaskContextSuite extends SparkFunSuite with BeforeAndAfter with LocalSpark if (stageAttemptNumber < 2) { // Throw FetchFailedException to explicitly trigger stage resubmission. A normal exception // will only trigger task resubmission in the same stage. - throw new FetchFailedException(null, 0, 0, 0, "Fake") + throw new FetchFailedException(null, 0, 0L, 0, 0, "Fake") } Seq(stageAttemptNumber).iterator }.collect() diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala index fedfa083e8d8f..5b1cb08aa4813 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -1262,7 +1262,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg // now fail those tasks tsmSpy.handleFailedTask(taskDescs(0).taskId, TaskState.FAILED, - FetchFailed(BlockManagerId(taskDescs(0).executorId, "host1", 12345), 0, 0, 0, "ignored")) + FetchFailed(BlockManagerId(taskDescs(0).executorId, "host1", 12345), 0, 0L, 0, 0, "ignored")) tsmSpy.handleFailedTask(taskDescs(1).taskId, TaskState.FAILED, ExecutorLostFailure(taskDescs(1).executorId, exitCausedByApp = false, reason = None)) tsmSpy.handleFailedTask(taskDescs(2).taskId, TaskState.FAILED, @@ -1302,7 +1302,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg // Fail the task with fetch failure tsm.handleFailedTask(taskDescs(0).taskId, TaskState.FAILED, - FetchFailed(BlockManagerId(taskDescs(0).executorId, "host1", 12345), 0, 0, 0, "ignored")) + FetchFailed(BlockManagerId(taskDescs(0).executorId, "host1", 12345), 0, 0L, 0, 0, "ignored")) assert(blacklistTracker.isNodeBlacklisted("host1")) } diff --git a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala index 1913b8d425519..5563da4a0b2e6 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala @@ -316,10 +316,12 @@ class UISeleniumSuite extends SparkFunSuite with WebBrowser with Matchers with B val env = SparkEnv.get val bmAddress = env.blockManager.blockManagerId val shuffleId = shuffleHandle.shuffleId - val mapId = 0 + val mapTaskId = 0L + val mapIndex = 0 val reduceId = taskContext.partitionId() val message = "Simulated fetch failure" - throw new FetchFailedException(bmAddress, shuffleId, mapId, reduceId, message) + throw new FetchFailedException( + bmAddress, shuffleId, mapTaskId, mapIndex, reduceId, message) } else { x } diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index 84d7084209ae9..9b2aae0e120ea 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -179,7 +179,7 @@ class JsonProtocolSuite extends SparkFunSuite { testJobResult(jobFailed) // TaskEndReason - val fetchFailed = FetchFailed(BlockManagerId("With or", "without you", 15), 17, 18, 19, + val fetchFailed = FetchFailed(BlockManagerId("With or", "without you", 15), 17, 16L, 18, 19, "Some exception") val fetchMetadataFailed = new MetadataFetchFailedException(17, 19, "metadata Fetch failed exception").toTaskFailedReason @@ -296,12 +296,12 @@ class JsonProtocolSuite extends SparkFunSuite { test("FetchFailed backwards compatibility") { // FetchFailed in Spark 1.1.0 does not have a "Message" property. - val fetchFailed = FetchFailed(BlockManagerId("With or", "without you", 15), 17, 18, 19, + val fetchFailed = FetchFailed(BlockManagerId("With or", "without you", 15), 17, 16L, 18, 19, "ignored") val oldEvent = JsonProtocol.taskEndReasonToJson(fetchFailed) .removeField({ _._1 == "Message" }) - val expectedFetchFailed = FetchFailed(BlockManagerId("With or", "without you", 15), 17, 18, 19, - "Unknown reason") + val expectedFetchFailed = FetchFailed(BlockManagerId("With or", "without you", 15), 17, 16L, + 18, 19, "Unknown reason") assert(expectedFetchFailed === JsonProtocol.taskEndReasonFromJson(oldEvent)) } @@ -731,6 +731,7 @@ private[spark] object JsonProtocolSuite extends Assertions { case (Resubmitted, Resubmitted) => case (r1: FetchFailed, r2: FetchFailed) => assert(r1.shuffleId === r2.shuffleId) + assert(r1.mapTaskId === r2.mapTaskId) assert(r1.mapIndex === r2.mapIndex) assert(r1.reduceId === r2.reduceId) assert(r1.bmAddress === r2.bmAddress) From 212b2012a064894935bb0855a4e6efb67a9e06b7 Mon Sep 17 00:00:00 2001 From: Yuanjian Li Date: Sun, 8 Sep 2019 11:15:30 +0800 Subject: [PATCH 09/15] fix mima --- project/MimaExcludes.scala | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index c2b9c96e4a05b..66f1eb7af5bad 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -436,7 +436,13 @@ object MimaExcludes { ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.storage.ShuffleIndexBlockId.mapId"), ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.storage.ShuffleDataBlockId.mapId"), ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.storage.ShuffleBlockId.mapId"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.FetchFailed.mapId") + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.FetchFailed.mapId"), + ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.FetchFailed$"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.FetchFailed.apply"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.FetchFailed.copy$default$5"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.FetchFailed.copy"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.FetchFailed.copy$default$3"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.FetchFailed.this"), ) // Exclude rules for 2.4.x From 0d9154411a1e945bbca13a3dbb9b8aa15ac21b51 Mon Sep 17 00:00:00 2001 From: Yuanjian Li Date: Mon, 9 Sep 2019 13:48:41 +0800 Subject: [PATCH 10/15] fix --- project/MimaExcludes.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 66f1eb7af5bad..1de47f8dec983 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -442,7 +442,7 @@ object MimaExcludes { ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.FetchFailed.copy$default$5"), ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.FetchFailed.copy"), ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.FetchFailed.copy$default$3"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.FetchFailed.this"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.FetchFailed.this") ) // Exclude rules for 2.4.x From da73b56add021fbc6c49d1fb27e249343d623045 Mon Sep 17 00:00:00 2001 From: Yuanjian Li Date: Mon, 16 Sep 2019 21:10:25 +0800 Subject: [PATCH 11/15] Address comment by squito and resolve conflict --- .../scala/org/apache/spark/MapOutputTracker.scala | 4 ++-- .../org/apache/spark/scheduler/DAGScheduler.scala | 15 +++++++++------ .../shuffle/sort/UnsafeShuffleWriterSuite.java | 4 ++-- .../spark/scheduler/DAGSchedulerSuite.scala | 6 +++--- 4 files changed, 16 insertions(+), 13 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 059c1b368570f..9e5b3716dfe0f 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -871,9 +871,9 @@ private[spark] object MapOutputTracker extends Logging { if (size != 0) { if (useOldFetchProtocol) { // While we use the old shuffle fetch protocol, we use mapIndex as mapTaskId in the - // ShuffleBlockId, so here set mapIndex with a invalid value -1. + // ShuffleBlockId. splitsByAddress.getOrElseUpdate(status.location, ListBuffer()) += - ((ShuffleBlockId(shuffleId, mapIndex, part), size, -1)) + ((ShuffleBlockId(shuffleId, mapIndex, part), size, mapIndex)) } else { splitsByAddress.getOrElseUpdate(status.location, ListBuffer()) += ((ShuffleBlockId(shuffleId, status.mapTaskId, part), size, mapIndex)) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index ca6e1847ce704..c9101e983bef7 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1104,10 +1104,10 @@ private[spark] class DAGScheduler( logDebug("submitMissingTasks(" + stage + ")") // Before find missing partition, do the intermediate state clean work first. - // The operation here can make sure for the intermediate stage, `findMissingPartitions()` - // returns all partitions every time. + // The operation here can make sure for the partially completed intermediate stage, + // `findMissingPartitions()` returns all partitions every time. stage match { - case sms: ShuffleMapStage if stage.isIndeterminate => + case sms: ShuffleMapStage if stage.isIndeterminate && !sms.isAvailable => mapOutputTracker.unregisterAllMapOutput(sms.shuffleDep.shuffleId) case _ => } @@ -1610,6 +1610,8 @@ private[spark] class DAGScheduler( activeJobs.foreach(job => collectStagesToRollback(job.finalStage :: Nil)) + // The stages will be rolled back after checking + val rollingBackStages = HashSet[Stage](mapStage) stagesToRollback.foreach { case mapStage: ShuffleMapStage => val numMissingPartitions = mapStage.findMissingPartitions().length @@ -1622,9 +1624,7 @@ private[spark] class DAGScheduler( "SPARK-27665 and SPARK-25341." abortStage(mapStage, reason, None) } else { - logInfo(s"The indeterminate stage $mapStage will be resubmitted," + - " the stage self and all indeterminate parent stage will be" + - " rollback and whole stage rerun.") + rollingBackStages += mapStage } } @@ -1637,6 +1637,9 @@ private[spark] class DAGScheduler( case _ => } + logInfo(s"The shuffle map stage $mapStage with indeterminate output was failed, " + + s"we will roll back and rerun below stages which include itself and all its " + + s"indeterminate child stages: $rollingBackStages") } // We expect one executor failure to trigger many FetchFailures in rapid succession, diff --git a/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java b/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java index ca5958658c2f8..244aa9a6baa9c 100644 --- a/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java +++ b/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java @@ -153,11 +153,11 @@ public void setUp() throws IOException { doAnswer(renameTempAnswer) .when(shuffleBlockResolver) - .writeIndexFileAndCommit(anyInt(), anyInt(), any(long[].class), any(File.class)); + .writeIndexFileAndCommit(anyInt(), anyLong(), any(long[].class), any(File.class)); doAnswer(renameTempAnswer) .when(shuffleBlockResolver) - .writeIndexFileAndCommit(anyInt(), anyLong(), any(long[].class), any(File.class)); + .writeIndexFileAndCommit(anyInt(), anyLong(), any(long[].class), eq(null)); when(diskBlockManager.createTempShuffleBlock()).thenAnswer(invocationOnMock -> { TempShuffleBlockId blockId = new TempShuffleBlockId(UUID.randomUUID()); diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 5800a2c85307d..a1d51cf429857 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -2891,9 +2891,9 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi scheduler.resubmitFailedStages() def checkAndCompleteRetryStage( - taskSetIndex: Int, - stageId: Int, - shuffleId: Int): Unit = { + taskSetIndex: Int, + stageId: Int, + shuffleId: Int): Unit = { assert(taskSets(taskSetIndex).stageId == stageId) assert(taskSets(taskSetIndex).stageAttemptId == 1) assert(taskSets(taskSetIndex).tasks.length == 2) From 69d59a1b9b41ba99968693c26d763ecff8ec0358 Mon Sep 17 00:00:00 2001 From: Yuanjian Li Date: Tue, 17 Sep 2019 16:22:11 +0800 Subject: [PATCH 12/15] Use mapId for user visable side --- .../api/ShuffleExecutorComponents.java | 10 ++++---- .../shuffle/sort/UnsafeShuffleWriter.java | 13 +++------- .../LocalDiskShuffleExecutorComponents.java | 4 ++-- .../io/LocalDiskShuffleMapOutputWriter.java | 10 ++++---- .../org/apache/spark/MapOutputTracker.scala | 2 +- .../org/apache/spark/TaskEndReason.scala | 4 ++-- .../spark/shuffle/FetchFailedException.scala | 4 ++-- .../shuffle/IndexShuffleBlockResolver.scala | 4 ++-- .../org/apache/spark/storage/BlockId.scala | 24 +++++++++---------- .../org/apache/spark/util/JsonProtocol.scala | 8 +++---- .../apache/spark/executor/ExecutorSuite.scala | 2 +- .../apache/spark/storage/BlockIdSuite.scala | 6 ++--- .../org/apache/spark/ui/UISeleniumSuite.scala | 4 ++-- .../apache/spark/util/JsonProtocolSuite.scala | 2 +- project/MimaExcludes.scala | 8 +++---- 15 files changed, 48 insertions(+), 57 deletions(-) diff --git a/core/src/main/java/org/apache/spark/shuffle/api/ShuffleExecutorComponents.java b/core/src/main/java/org/apache/spark/shuffle/api/ShuffleExecutorComponents.java index a0fd415cc149c..042d0e5ae2b46 100644 --- a/core/src/main/java/org/apache/spark/shuffle/api/ShuffleExecutorComponents.java +++ b/core/src/main/java/org/apache/spark/shuffle/api/ShuffleExecutorComponents.java @@ -41,13 +41,13 @@ public interface ShuffleExecutorComponents { * Called once per map task to create a writer that will be responsible for persisting all the * partitioned bytes written by that map task. * @param shuffleId Unique identifier for the shuffle the map task is a part of - * @param mapTaskId An ID of the map task. The ID is unique within this Spark application. + * @param mapId An ID of the map task. The ID is unique within this Spark application. * @param numPartitions The number of partitions that will be written by the map task. Some of * these partitions may be empty. */ ShuffleMapOutputWriter createMapOutputWriter( int shuffleId, - long mapTaskId, + long mapId, int numPartitions) throws IOException; /** @@ -59,13 +59,11 @@ ShuffleMapOutputWriter createMapOutputWriter( * preserving an optimization in the local disk shuffle storage implementation. * * @param shuffleId Unique identifier for the shuffle the map task is a part of - * @param mapTaskAttemptId Identifier of the task attempt. Multiple attempts of the same map task - * with the same (shuffleId, mapId) pair can be distinguished by the - * different values of mapTaskAttemptId. + * @param mapId An ID of the map task. The ID is unique within this Spark application. */ default Optional createSingleFileMapOutputWriter( int shuffleId, - long mapTaskAttemptId) throws IOException { + long mapId) throws IOException { return Optional.empty(); } } diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java index 2dc14ac2e7f6f..e8e7c0e3fc9fa 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java @@ -264,15 +264,11 @@ private long[] mergeSpills(SpillInfo[] spills) throws IOException { long[] partitionLengths; if (spills.length == 0) { final ShuffleMapOutputWriter mapWriter = shuffleExecutorComponents - .createMapOutputWriter( - shuffleId, - taskContext.taskAttemptId(), - partitioner.numPartitions()); + .createMapOutputWriter(shuffleId, mapTaskId, partitioner.numPartitions()); return mapWriter.commitAllPartitions(); } else if (spills.length == 1) { Optional maybeSingleFileWriter = - shuffleExecutorComponents.createSingleFileMapOutputWriter( - shuffleId, taskContext.taskAttemptId()); + shuffleExecutorComponents.createSingleFileMapOutputWriter(shuffleId, mapTaskId); if (maybeSingleFileWriter.isPresent()) { // Here, we don't need to perform any metrics updates because the bytes written to this // output file would have already been counted as shuffle bytes written. @@ -297,10 +293,7 @@ private long[] mergeSpillsUsingStandardWriter(SpillInfo[] spills) throws IOExcep CompressionCodec$.MODULE$.supportsConcatenationOfSerializedStreams(compressionCodec); final boolean encryptionEnabled = blockManager.serializerManager().encryptionEnabled(); final ShuffleMapOutputWriter mapWriter = shuffleExecutorComponents - .createMapOutputWriter( - shuffleId, - taskContext.taskAttemptId(), - partitioner.numPartitions()); + .createMapOutputWriter(shuffleId, mapTaskId, partitioner.numPartitions()); try { // There are multiple spills to merge, so none of these spill files' lengths were counted // towards our shuffle write count or shuffle write time. If we use the slow merge path, diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleExecutorComponents.java b/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleExecutorComponents.java index 94b5c9180ff85..a0c7d3c248d48 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleExecutorComponents.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleExecutorComponents.java @@ -61,14 +61,14 @@ public void initializeExecutor(String appId, String execId) { @Override public ShuffleMapOutputWriter createMapOutputWriter( int shuffleId, - long mapTaskId, + long mapId, int numPartitions) { if (blockResolver == null) { throw new IllegalStateException( "Executor components must be initialized before getting writers."); } return new LocalDiskShuffleMapOutputWriter( - shuffleId, mapTaskId, numPartitions, blockResolver, sparkConf); + shuffleId, mapId, numPartitions, blockResolver, sparkConf); } @Override diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriter.java index 559ff915cdfd1..a6529fd76188a 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriter.java @@ -48,7 +48,7 @@ public class LocalDiskShuffleMapOutputWriter implements ShuffleMapOutputWriter { LoggerFactory.getLogger(LocalDiskShuffleMapOutputWriter.class); private final int shuffleId; - private final long mapTaskId; + private final long mapId; private final IndexShuffleBlockResolver blockResolver; private final long[] partitionLengths; private final int bufferSize; @@ -64,18 +64,18 @@ public class LocalDiskShuffleMapOutputWriter implements ShuffleMapOutputWriter { public LocalDiskShuffleMapOutputWriter( int shuffleId, - long mapTaskId, + long mapId, int numPartitions, IndexShuffleBlockResolver blockResolver, SparkConf sparkConf) { this.shuffleId = shuffleId; - this.mapTaskId = mapTaskId; + this.mapId = mapId; this.blockResolver = blockResolver; this.bufferSize = (int) (long) sparkConf.get( package$.MODULE$.SHUFFLE_UNSAFE_FILE_OUTPUT_BUFFER_SIZE()) * 1024; this.partitionLengths = new long[numPartitions]; - this.outputFile = blockResolver.getDataFile(shuffleId, mapTaskId); + this.outputFile = blockResolver.getDataFile(shuffleId, mapId); this.outputTempFile = null; } @@ -112,7 +112,7 @@ public long[] commitAllPartitions() throws IOException { } cleanUp(); File resolvedTmp = outputTempFile != null && outputTempFile.isFile() ? outputTempFile : null; - blockResolver.writeIndexFileAndCommit(shuffleId, mapTaskId, partitionLengths, resolvedTmp); + blockResolver.writeIndexFileAndCommit(shuffleId, mapId, partitionLengths, resolvedTmp); return partitionLengths; } diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 9e5b3716dfe0f..53329f0a937bd 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -870,7 +870,7 @@ private[spark] object MapOutputTracker extends Logging { val size = status.getSizeForBlock(part) if (size != 0) { if (useOldFetchProtocol) { - // While we use the old shuffle fetch protocol, we use mapIndex as mapTaskId in the + // While we use the old shuffle fetch protocol, we use mapIndex as mapId in the // ShuffleBlockId. splitsByAddress.getOrElseUpdate(status.location, ListBuffer()) += ((ShuffleBlockId(shuffleId, mapIndex, part), size, mapIndex)) diff --git a/core/src/main/scala/org/apache/spark/TaskEndReason.scala b/core/src/main/scala/org/apache/spark/TaskEndReason.scala index 199d7f41349e6..b13028f868072 100644 --- a/core/src/main/scala/org/apache/spark/TaskEndReason.scala +++ b/core/src/main/scala/org/apache/spark/TaskEndReason.scala @@ -83,7 +83,7 @@ case object Resubmitted extends TaskFailedReason { case class FetchFailed( bmAddress: BlockManagerId, // Note that bmAddress can be null shuffleId: Int, - mapTaskId: Long, + mapId: Long, mapIndex: Int, reduceId: Int, message: String) @@ -91,7 +91,7 @@ case class FetchFailed( override def toErrorString: String = { val bmAddressString = if (bmAddress == null) "null" else bmAddress.toString s"FetchFailed($bmAddressString, shuffleId=$shuffleId, mapIndex=$mapIndex, " + - s"mapTaskId=$mapTaskId, reduceId=$reduceId, message=\n$message\n)" + s"mapId=$mapId, reduceId=$reduceId, message=\n$message\n)" } /** diff --git a/core/src/main/scala/org/apache/spark/shuffle/FetchFailedException.scala b/core/src/main/scala/org/apache/spark/shuffle/FetchFailedException.scala index 965546ca9fb5f..6509a04dc4893 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/FetchFailedException.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/FetchFailedException.scala @@ -35,7 +35,7 @@ import org.apache.spark.util.Utils private[spark] class FetchFailedException( bmAddress: BlockManagerId, shuffleId: Int, - mapTaskId: Long, + mapId: Long, mapIndex: Int, reduceId: Int, message: String, @@ -59,7 +59,7 @@ private[spark] class FetchFailedException( Option(TaskContext.get()).map(_.setFetchFailed(this)) def toTaskFailedReason: TaskFailedReason = FetchFailed( - bmAddress, shuffleId, mapTaskId, mapIndex, reduceId, Utils.exceptionString(this)) + bmAddress, shuffleId, mapId, mapIndex, reduceId, Utils.exceptionString(this)) } /** diff --git a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala index 1a4b9cdad2adb..e177a443237de 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala @@ -194,7 +194,7 @@ private[spark] class IndexShuffleBlockResolver( override def getBlockData(blockId: ShuffleBlockId): ManagedBuffer = { // The block is actually going to be a range of a single map output file for this map, so // find out the consolidated file, then the offset within that from our index - val indexFile = getIndexFile(blockId.shuffleId, blockId.mapTaskId) + val indexFile = getIndexFile(blockId.shuffleId, blockId.mapId) // SPARK-22982: if this FileInputStream's position is seeked forward by another piece of code // which is incorrectly using our file descriptor then this code will fetch the wrong offsets @@ -216,7 +216,7 @@ private[spark] class IndexShuffleBlockResolver( } new FileSegmentManagedBuffer( transportConf, - getDataFile(blockId.shuffleId, blockId.mapTaskId), + getDataFile(blockId.shuffleId, blockId.mapId), offset, nextOffset - offset) } finally { diff --git a/core/src/main/scala/org/apache/spark/storage/BlockId.scala b/core/src/main/scala/org/apache/spark/storage/BlockId.scala index 71424f89e4aaf..9c5b7f64e7abe 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockId.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockId.scala @@ -52,18 +52,18 @@ case class RDDBlockId(rddId: Int, splitIndex: Int) extends BlockId { // Format of the shuffle block ids (including data and index) should be kept in sync with // org.apache.spark.network.shuffle.ExternalShuffleBlockResolver#getBlockData(). @DeveloperApi -case class ShuffleBlockId(shuffleId: Int, mapTaskId: Long, reduceId: Int) extends BlockId { - override def name: String = "shuffle_" + shuffleId + "_" + mapTaskId + "_" + reduceId +case class ShuffleBlockId(shuffleId: Int, mapId: Long, reduceId: Int) extends BlockId { + override def name: String = "shuffle_" + shuffleId + "_" + mapId + "_" + reduceId } @DeveloperApi -case class ShuffleDataBlockId(shuffleId: Int, mapTaskId: Long, reduceId: Int) extends BlockId { - override def name: String = "shuffle_" + shuffleId + "_" + mapTaskId + "_" + reduceId + ".data" +case class ShuffleDataBlockId(shuffleId: Int, mapId: Long, reduceId: Int) extends BlockId { + override def name: String = "shuffle_" + shuffleId + "_" + mapId + "_" + reduceId + ".data" } @DeveloperApi -case class ShuffleIndexBlockId(shuffleId: Int, mapTaskId: Long, reduceId: Int) extends BlockId { - override def name: String = "shuffle_" + shuffleId + "_" + mapTaskId + "_" + reduceId + ".index" +case class ShuffleIndexBlockId(shuffleId: Int, mapId: Long, reduceId: Int) extends BlockId { + override def name: String = "shuffle_" + shuffleId + "_" + mapId + "_" + reduceId + ".index" } @DeveloperApi @@ -116,12 +116,12 @@ object BlockId { def apply(name: String): BlockId = name match { case RDD(rddId, splitIndex) => RDDBlockId(rddId.toInt, splitIndex.toInt) - case SHUFFLE(shuffleId, mapTaskId, reduceId) => - ShuffleBlockId(shuffleId.toInt, mapTaskId.toLong, reduceId.toInt) - case SHUFFLE_DATA(shuffleId, mapTaskId, reduceId) => - ShuffleDataBlockId(shuffleId.toInt, mapTaskId.toLong, reduceId.toInt) - case SHUFFLE_INDEX(shuffleId, mapTaskId, reduceId) => - ShuffleIndexBlockId(shuffleId.toInt, mapTaskId.toLong, reduceId.toInt) + case SHUFFLE(shuffleId, mapId, reduceId) => + ShuffleBlockId(shuffleId.toInt, mapId.toLong, reduceId.toInt) + case SHUFFLE_DATA(shuffleId, mapId, reduceId) => + ShuffleDataBlockId(shuffleId.toInt, mapId.toLong, reduceId.toInt) + case SHUFFLE_INDEX(shuffleId, mapId, reduceId) => + ShuffleIndexBlockId(shuffleId.toInt, mapId.toLong, reduceId.toInt) case BROADCAST(broadcastId, field) => BroadcastBlockId(broadcastId.toLong, field.stripPrefix("_")) case TASKRESULT(taskId) => diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index c3e501d61ab52..353590d201bc5 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -419,7 +419,7 @@ private[spark] object JsonProtocol { map(blockManagerIdToJson).getOrElse(JNothing) ("Block Manager Address" -> blockManagerAddress) ~ ("Shuffle ID" -> fetchFailed.shuffleId) ~ - ("Map Task ID" -> fetchFailed.mapTaskId) ~ + ("Map ID" -> fetchFailed.mapId) ~ ("Map Index" -> fetchFailed.mapIndex) ~ ("Reduce ID" -> fetchFailed.reduceId) ~ ("Message" -> fetchFailed.message) @@ -975,12 +975,12 @@ private[spark] object JsonProtocol { case `fetchFailed` => val blockManagerAddress = blockManagerIdFromJson(json \ "Block Manager Address") val shuffleId = (json \ "Shuffle ID").extract[Int] - val mapTaskId = (json \ "Map Task ID").extract[Long] + val mapId = (json \ "Map ID").extract[Long] val mapIndex = (json \ "Map Index").extract[Int] val reduceId = (json \ "Reduce ID").extract[Int] val message = jsonOption(json \ "Message").map(_.extract[String]) - new FetchFailed(blockManagerAddress, shuffleId, mapTaskId, mapIndex, - reduceId, message.getOrElse("Unknown reason")) + new FetchFailed(blockManagerAddress, shuffleId, mapId, mapIndex, reduceId, + message.getOrElse("Unknown reason")) case `exceptionFailure` => val className = (json \ "Class Name").extract[String] val description = (json \ "Description").extract[String] diff --git a/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala b/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala index b5f563b2aeeea..3faab52d6510c 100644 --- a/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala +++ b/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala @@ -528,7 +528,7 @@ class FetchFailureThrowingRDD(sc: SparkContext) extends RDD[Int](sc, Nil) { throw new FetchFailedException( bmAddress = BlockManagerId("1", "hostA", 1234), shuffleId = 0, - mapTaskId = 0L, + mapId = 0L, mapIndex = 0, reduceId = 0, message = "fake fetch failure" diff --git a/core/src/test/scala/org/apache/spark/storage/BlockIdSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockIdSuite.scala index 8d6de10daa5b1..ff4755833a916 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockIdSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockIdSuite.scala @@ -58,7 +58,7 @@ class BlockIdSuite extends SparkFunSuite { assert(id.name === "shuffle_1_2_3") assert(id.asRDDId === None) assert(id.shuffleId === 1) - assert(id.mapTaskId === 2) + assert(id.mapId === 2) assert(id.reduceId === 3) assert(id.isShuffle) assertSame(id, BlockId(id.toString)) @@ -71,7 +71,7 @@ class BlockIdSuite extends SparkFunSuite { assert(id.name === "shuffle_4_5_6.data") assert(id.asRDDId === None) assert(id.shuffleId === 4) - assert(id.mapTaskId === 5) + assert(id.mapId === 5) assert(id.reduceId === 6) assert(!id.isShuffle) assertSame(id, BlockId(id.toString)) @@ -84,7 +84,7 @@ class BlockIdSuite extends SparkFunSuite { assert(id.name === "shuffle_7_8_9.index") assert(id.asRDDId === None) assert(id.shuffleId === 7) - assert(id.mapTaskId === 8) + assert(id.mapId === 8) assert(id.reduceId === 9) assert(!id.isShuffle) assertSame(id, BlockId(id.toString)) diff --git a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala index 5563da4a0b2e6..580af086ba9da 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala @@ -316,12 +316,12 @@ class UISeleniumSuite extends SparkFunSuite with WebBrowser with Matchers with B val env = SparkEnv.get val bmAddress = env.blockManager.blockManagerId val shuffleId = shuffleHandle.shuffleId - val mapTaskId = 0L + val mapId = 0L val mapIndex = 0 val reduceId = taskContext.partitionId() val message = "Simulated fetch failure" throw new FetchFailedException( - bmAddress, shuffleId, mapTaskId, mapIndex, reduceId, message) + bmAddress, shuffleId, mapId, mapIndex, reduceId, message) } else { x } diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index 9b2aae0e120ea..54625a93679fb 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -731,7 +731,7 @@ private[spark] object JsonProtocolSuite extends Assertions { case (Resubmitted, Resubmitted) => case (r1: FetchFailed, r2: FetchFailed) => assert(r1.shuffleId === r2.shuffleId) - assert(r1.mapTaskId === r2.mapTaskId) + assert(r1.mapId === r2.mapId) assert(r1.mapIndex === r2.mapIndex) assert(r1.reduceId === r2.reduceId) assert(r1.bmAddress === r2.bmAddress) diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 1de47f8dec983..9f60338df7059 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -433,10 +433,10 @@ object MimaExcludes { ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.storage.ShuffleIndexBlockId.apply"), ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.storage.ShuffleDataBlockId.apply"), ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.storage.ShuffleBlockId.apply"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.storage.ShuffleIndexBlockId.mapId"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.storage.ShuffleDataBlockId.mapId"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.storage.ShuffleBlockId.mapId"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.FetchFailed.mapId"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.storage.ShuffleIndexBlockId.mapId"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.storage.ShuffleDataBlockId.mapId"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.storage.ShuffleBlockId.mapId"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.FetchFailed.mapId"), ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.FetchFailed$"), ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.FetchFailed.apply"), ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.FetchFailed.copy$default$5"), From c86f6cc4d0fded370621e73e4ac910e8f1b01dd3 Mon Sep 17 00:00:00 2001 From: Yuanjian Li Date: Wed, 18 Sep 2019 23:56:34 +0800 Subject: [PATCH 13/15] further rename --- .../network/shuffle/ExternalBlockHandler.java | 14 +++++----- .../shuffle/ExternalShuffleBlockResolver.java | 19 +++++++------- .../shuffle/OneForOneBlockFetcher.java | 22 ++++++++-------- .../shuffle/protocol/FetchShuffleBlocks.java | 26 +++++++++---------- .../sort/BypassMergeSortShuffleWriter.java | 12 ++++----- .../shuffle/sort/UnsafeShuffleWriter.java | 12 ++++----- .../network/netty/NettyBlockRpcServer.scala | 11 ++++---- .../shuffle/IndexShuffleBlockResolver.scala | 21 +++++++-------- .../storage/ShuffleBlockFetcherIterator.scala | 4 +-- .../util/collection/ExternalSorter.scala | 6 ++--- .../BlockStoreShuffleReaderSuite.scala | 12 ++++----- .../BypassMergeSortShuffleWriterSuite.scala | 8 +++--- .../sort/IndexShuffleBlockResolverSuite.scala | 12 ++++----- 13 files changed, 88 insertions(+), 91 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 1d8303c1aa3f5..2d7a72315cf23 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 @@ -321,30 +321,30 @@ private class ShuffleManagedBufferIterator implements Iterator { private final String appId; private final String execId; private final int shuffleId; - private final long[] mapTaskIds; + private final long[] mapIds; private final int[][] reduceIds; ShuffleManagedBufferIterator(FetchShuffleBlocks msg) { appId = msg.appId; execId = msg.execId; shuffleId = msg.shuffleId; - mapTaskIds = msg.mapTaskIds; + mapIds = msg.mapIds; reduceIds = msg.reduceIds; } @Override public boolean hasNext() { - // mapTaskIds.length must equal to reduceIds.length, and the passed in FetchShuffleBlocks - // must have non-empty mapTaskIds and reduceIds, see the checking logic in + // mapIds.length must equal to reduceIds.length, and the passed in FetchShuffleBlocks + // must have non-empty mapIds and reduceIds, see the checking logic in // OneForOneBlockFetcher. - assert(mapTaskIds.length != 0 && mapTaskIds.length == reduceIds.length); - return mapIdx < mapTaskIds.length && reduceIdx < reduceIds[mapIdx].length; + assert(mapIds.length != 0 && mapIds.length == reduceIds.length); + return mapIdx < mapIds.length && reduceIdx < reduceIds[mapIdx].length; } @Override public ManagedBuffer next() { final ManagedBuffer block = blockManager.getBlockData( - appId, execId, shuffleId, mapTaskIds[mapIdx], reduceIds[mapIdx][reduceIdx]); + appId, execId, shuffleId, mapIds[mapIdx], reduceIds[mapIdx][reduceIdx]); if (reduceIdx < reduceIds[mapIdx].length - 1) { reduceIdx += 1; } else { 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 e51b98fb7c8d3..8b0d1e145a813 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 @@ -165,21 +165,21 @@ public void registerExecutor( } /** - * Obtains a FileSegmentManagedBuffer from (shuffleId, mapTaskId, reduceId). We make assumptions + * Obtains a FileSegmentManagedBuffer from (shuffleId, mapId, reduceId). We make assumptions * about how the hash and sort based shuffles store their data. */ public ManagedBuffer getBlockData( String appId, String execId, int shuffleId, - long mapTaskId, + long mapId, int reduceId) { ExecutorShuffleInfo executor = executors.get(new AppExecId(appId, execId)); if (executor == null) { throw new RuntimeException( String.format("Executor is not registered (appId=%s, execId=%s)", appId, execId)); } - return getSortBasedShuffleBlockData(executor, shuffleId, mapTaskId, reduceId); + return getSortBasedShuffleBlockData(executor, shuffleId, mapId, reduceId); } public ManagedBuffer getRddBlockData( @@ -291,15 +291,14 @@ private void deleteNonShuffleServiceServedFiles(String[] dirs) { } /** - * Sort-based shuffle data uses an index called "shuffle_ShuffleId_MapTaskId_0.index" into a data - * file called "shuffle_ShuffleId_MapTaskId_0.data". - * This logic is from IndexShuffleBlockResolver, and the block id format is from - * ShuffleDataBlockId and ShuffleIndexBlockId. + * Sort-based shuffle data uses an index called "shuffle_ShuffleId_MapId_0.index" into a data file + * called "shuffle_ShuffleId_MapId_0.data". This logic is from IndexShuffleBlockResolver, + * and the block id format is from ShuffleDataBlockId and ShuffleIndexBlockId. */ private ManagedBuffer getSortBasedShuffleBlockData( - ExecutorShuffleInfo executor, int shuffleId, long mapTaskId, int reduceId) { + ExecutorShuffleInfo executor, int shuffleId, long mapId, int reduceId) { File indexFile = ExecutorDiskUtils.getFile(executor.localDirs, executor.subDirsPerLocalDir, - "shuffle_" + shuffleId + "_" + mapTaskId + "_0.index"); + "shuffle_" + shuffleId + "_" + mapId + "_0.index"); try { ShuffleIndexInformation shuffleIndexInformation = shuffleIndexCache.get(indexFile); @@ -307,7 +306,7 @@ private ManagedBuffer getSortBasedShuffleBlockData( return new FileSegmentManagedBuffer( conf, ExecutorDiskUtils.getFile(executor.localDirs, executor.subDirsPerLocalDir, - "shuffle_" + shuffleId + "_" + mapTaskId + "_0.data"), + "shuffle_" + shuffleId + "_" + mapId + "_0.data"), shuffleIndexRecord.getOffset(), shuffleIndexRecord.getLength()); } catch (ExecutionException e) { diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java index 446014b775bcf..52854c86be3e6 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java @@ -108,7 +108,7 @@ private boolean isShuffleBlocks(String[] blockIds) { /** * Analyze the pass in blockIds and create FetchShuffleBlocks message. - * The blockIds has been sorted by mapTaskId and reduceId. It's produced in + * The blockIds has been sorted by mapId and reduceId. It's produced in * org.apache.spark.MapOutputTracker.convertMapStatuses. */ private FetchShuffleBlocks createFetchShuffleBlocksMsg( @@ -121,21 +121,21 @@ private FetchShuffleBlocks createFetchShuffleBlocksMsg( throw new IllegalArgumentException("Expected shuffleId=" + shuffleId + ", got:" + blockId); } - long mapTaskId = blockIdParts.middle; - if (!mapIdToReduceIds.containsKey(mapTaskId)) { - mapIdToReduceIds.put(mapTaskId, new ArrayList<>()); + long mapId = blockIdParts.middle; + if (!mapIdToReduceIds.containsKey(mapId)) { + mapIdToReduceIds.put(mapId, new ArrayList<>()); } - mapIdToReduceIds.get(mapTaskId).add(blockIdParts.right); + mapIdToReduceIds.get(mapId).add(blockIdParts.right); } - long[] mapTaskIds = Longs.toArray(mapIdToReduceIds.keySet()); - int[][] reduceIdArr = new int[mapTaskIds.length][]; - for (int i = 0; i < mapTaskIds.length; i++) { - reduceIdArr[i] = Ints.toArray(mapIdToReduceIds.get(mapTaskIds[i])); + long[] mapIds = Longs.toArray(mapIdToReduceIds.keySet()); + int[][] reduceIdArr = new int[mapIds.length][]; + for (int i = 0; i < mapIds.length; i++) { + reduceIdArr[i] = Ints.toArray(mapIdToReduceIds.get(mapIds[i])); } - return new FetchShuffleBlocks(appId, execId, shuffleId, mapTaskIds, reduceIdArr); + return new FetchShuffleBlocks(appId, execId, shuffleId, mapIds, reduceIdArr); } - /** Split the shuffleBlockId and return shuffleId, mapTaskId and reduceId. */ + /** Split the shuffleBlockId and return shuffleId, mapId and reduceId. */ private ImmutableTriple splitBlockId(String blockId) { String[] blockIdParts = blockId.split("_"); if (blockIdParts.length != 4 || !blockIdParts[0].equals("shuffle")) { diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/FetchShuffleBlocks.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/FetchShuffleBlocks.java index 63fc47ff9f03c..faa960d414bcc 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/FetchShuffleBlocks.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/FetchShuffleBlocks.java @@ -32,23 +32,23 @@ public class FetchShuffleBlocks extends BlockTransferMessage { public final String appId; public final String execId; public final int shuffleId; - // The length of mapTaskIds must equal to reduceIds.size(), for the i-th mapTaskId in mapTaskIds, - // it corresponds to the i-th int[] in reduceIds, which contains all reduce id for this map task. - public final long[] mapTaskIds; + // The length of mapIds must equal to reduceIds.size(), for the i-th mapId in mapIds, + // it corresponds to the i-th int[] in reduceIds, which contains all reduce id for this map id. + public final long[] mapIds; public final int[][] reduceIds; public FetchShuffleBlocks( String appId, String execId, int shuffleId, - long[] mapTaskIds, + long[] mapIds, int[][] reduceIds) { this.appId = appId; this.execId = execId; this.shuffleId = shuffleId; - this.mapTaskIds = mapTaskIds; + this.mapIds = mapIds; this.reduceIds = reduceIds; - assert(mapTaskIds.length == reduceIds.length); + assert(mapIds.length == reduceIds.length); } @Override @@ -60,7 +60,7 @@ public String toString() { .add("appId", appId) .add("execId", execId) .add("shuffleId", shuffleId) - .add("mapTaskIds", Arrays.toString(mapTaskIds)) + .add("mapIds", Arrays.toString(mapIds)) .add("reduceIds", Arrays.deepToString(reduceIds)) .toString(); } @@ -75,7 +75,7 @@ public boolean equals(Object o) { if (shuffleId != that.shuffleId) return false; if (!appId.equals(that.appId)) return false; if (!execId.equals(that.execId)) return false; - if (!Arrays.equals(mapTaskIds, that.mapTaskIds)) return false; + if (!Arrays.equals(mapIds, that.mapIds)) return false; return Arrays.deepEquals(reduceIds, that.reduceIds); } @@ -84,7 +84,7 @@ public int hashCode() { int result = appId.hashCode(); result = 31 * result + execId.hashCode(); result = 31 * result + shuffleId; - result = 31 * result + Arrays.hashCode(mapTaskIds); + result = 31 * result + Arrays.hashCode(mapIds); result = 31 * result + Arrays.deepHashCode(reduceIds); return result; } @@ -98,7 +98,7 @@ public int encodedLength() { return Encoders.Strings.encodedLength(appId) + Encoders.Strings.encodedLength(execId) + 4 /* encoded length of shuffleId */ - + Encoders.LongArrays.encodedLength(mapTaskIds) + + Encoders.LongArrays.encodedLength(mapIds) + 4 /* encoded length of reduceIds.size() */ + encodedLengthOfReduceIds; } @@ -108,7 +108,7 @@ public void encode(ByteBuf buf) { Encoders.Strings.encode(buf, appId); Encoders.Strings.encode(buf, execId); buf.writeInt(shuffleId); - Encoders.LongArrays.encode(buf, mapTaskIds); + Encoders.LongArrays.encode(buf, mapIds); buf.writeInt(reduceIds.length); for (int[] ids: reduceIds) { Encoders.IntArrays.encode(buf, ids); @@ -119,12 +119,12 @@ public static FetchShuffleBlocks decode(ByteBuf buf) { String appId = Encoders.Strings.decode(buf); String execId = Encoders.Strings.decode(buf); int shuffleId = buf.readInt(); - long[] mapTaskIds = Encoders.LongArrays.decode(buf); + long[] mapIds = Encoders.LongArrays.decode(buf); int reduceIdsSize = buf.readInt(); int[][] reduceIds = new int[reduceIdsSize][]; for (int i = 0; i < reduceIdsSize; i++) { reduceIds[i] = Encoders.IntArrays.decode(buf); } - return new FetchShuffleBlocks(appId, execId, shuffleId, mapTaskIds, reduceIds); + return new FetchShuffleBlocks(appId, execId, shuffleId, mapIds, reduceIds); } } diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java index ae0a8ca05c5b9..dc157eaa3b253 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java @@ -85,7 +85,7 @@ final class BypassMergeSortShuffleWriter extends ShuffleWriter { private final Partitioner partitioner; private final ShuffleWriteMetricsReporter writeMetrics; private final int shuffleId; - private final long mapTaskId; + private final long mapId; private final Serializer serializer; private final ShuffleExecutorComponents shuffleExecutorComponents; @@ -105,7 +105,7 @@ final class BypassMergeSortShuffleWriter extends ShuffleWriter { BypassMergeSortShuffleWriter( BlockManager blockManager, BypassMergeSortShuffleHandle handle, - long mapTaskId, + long mapId, SparkConf conf, ShuffleWriteMetricsReporter writeMetrics, ShuffleExecutorComponents shuffleExecutorComponents) { @@ -114,7 +114,7 @@ final class BypassMergeSortShuffleWriter extends ShuffleWriter { this.transferToEnabled = conf.getBoolean("spark.file.transferTo", true); this.blockManager = blockManager; final ShuffleDependency dep = handle.dependency(); - this.mapTaskId = mapTaskId; + this.mapId = mapId; this.shuffleId = dep.shuffleId(); this.partitioner = dep.partitioner(); this.numPartitions = partitioner.numPartitions(); @@ -127,12 +127,12 @@ final class BypassMergeSortShuffleWriter extends ShuffleWriter { public void write(Iterator> records) throws IOException { assert (partitionWriters == null); ShuffleMapOutputWriter mapOutputWriter = shuffleExecutorComponents - .createMapOutputWriter(shuffleId, mapTaskId, numPartitions); + .createMapOutputWriter(shuffleId, mapId, numPartitions); try { if (!records.hasNext()) { partitionLengths = mapOutputWriter.commitAllPartitions(); mapStatus = MapStatus$.MODULE$.apply( - blockManager.shuffleServerId(), partitionLengths, mapTaskId); + blockManager.shuffleServerId(), partitionLengths, mapId); return; } final SerializerInstance serInstance = serializer.newInstance(); @@ -166,7 +166,7 @@ public void write(Iterator> records) throws IOException { partitionLengths = writePartitionedData(mapOutputWriter); mapStatus = MapStatus$.MODULE$.apply( - blockManager.shuffleServerId(), partitionLengths, mapTaskId); + blockManager.shuffleServerId(), partitionLengths, mapId); } catch (Exception e) { try { mapOutputWriter.abort(e); diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java index e8e7c0e3fc9fa..2070369774cd5 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java @@ -78,7 +78,7 @@ public class UnsafeShuffleWriter extends ShuffleWriter { private final ShuffleWriteMetricsReporter writeMetrics; private final ShuffleExecutorComponents shuffleExecutorComponents; private final int shuffleId; - private final long mapTaskId; + private final long mapId; private final TaskContext taskContext; private final SparkConf sparkConf; private final boolean transferToEnabled; @@ -122,7 +122,7 @@ public UnsafeShuffleWriter( } this.blockManager = blockManager; this.memoryManager = memoryManager; - this.mapTaskId = taskContext.taskAttemptId(); + this.mapId = taskContext.taskAttemptId(); final ShuffleDependency dep = handle.dependency(); this.shuffleId = dep.shuffleId(); this.serializer = dep.serializer().newInstance(); @@ -228,7 +228,7 @@ void closeAndWriteOutput() throws IOException { } } mapStatus = MapStatus$.MODULE$.apply( - blockManager.shuffleServerId(), partitionLengths, mapTaskId); + blockManager.shuffleServerId(), partitionLengths, mapId); } @VisibleForTesting @@ -264,11 +264,11 @@ private long[] mergeSpills(SpillInfo[] spills) throws IOException { long[] partitionLengths; if (spills.length == 0) { final ShuffleMapOutputWriter mapWriter = shuffleExecutorComponents - .createMapOutputWriter(shuffleId, mapTaskId, partitioner.numPartitions()); + .createMapOutputWriter(shuffleId, mapId, partitioner.numPartitions()); return mapWriter.commitAllPartitions(); } else if (spills.length == 1) { Optional maybeSingleFileWriter = - shuffleExecutorComponents.createSingleFileMapOutputWriter(shuffleId, mapTaskId); + shuffleExecutorComponents.createSingleFileMapOutputWriter(shuffleId, mapId); if (maybeSingleFileWriter.isPresent()) { // Here, we don't need to perform any metrics updates because the bytes written to this // output file would have already been counted as shuffle bytes written. @@ -293,7 +293,7 @@ private long[] mergeSpillsUsingStandardWriter(SpillInfo[] spills) throws IOExcep CompressionCodec$.MODULE$.supportsConcatenationOfSerializedStreams(compressionCodec); final boolean encryptionEnabled = blockManager.serializerManager().encryptionEnabled(); final ShuffleMapOutputWriter mapWriter = shuffleExecutorComponents - .createMapOutputWriter(shuffleId, mapTaskId, partitioner.numPartitions()); + .createMapOutputWriter(shuffleId, mapId, partitioner.numPartitions()); try { // There are multiple spills to merge, so none of these spill files' lengths were counted // towards our shuffle write count or shuffle write time. If we use the slow merge path, diff --git a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala index 6e807eed503cc..b2ab31488e4c1 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala +++ b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala @@ -64,12 +64,11 @@ class NettyBlockRpcServer( responseContext.onSuccess(new StreamHandle(streamId, blocksNum).toByteBuffer) case fetchShuffleBlocks: FetchShuffleBlocks => - val blocks = fetchShuffleBlocks.mapTaskIds.zipWithIndex.flatMap { - case (mapTaskId, index) => - fetchShuffleBlocks.reduceIds.apply(index).map { reduceId => - blockManager.getBlockData( - ShuffleBlockId(fetchShuffleBlocks.shuffleId, mapTaskId, reduceId)) - } + val blocks = fetchShuffleBlocks.mapIds.zipWithIndex.flatMap { case (mapId, index) => + fetchShuffleBlocks.reduceIds.apply(index).map { reduceId => + blockManager.getBlockData( + ShuffleBlockId(fetchShuffleBlocks.shuffleId, mapId, reduceId)) + } } val numBlockIds = fetchShuffleBlocks.reduceIds.map(_.length).sum val streamId = streamManager.registerStream(appId, blocks.iterator.asJava, diff --git a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala index e177a443237de..332164a7be3e7 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala @@ -51,27 +51,26 @@ private[spark] class IndexShuffleBlockResolver( private val transportConf = SparkTransportConf.fromSparkConf(conf, "shuffle") - def getDataFile(shuffleId: Int, mapTaskId: Long): File = { - blockManager.diskBlockManager.getFile(ShuffleDataBlockId(shuffleId, mapTaskId, NOOP_REDUCE_ID)) + def getDataFile(shuffleId: Int, mapId: Long): File = { + blockManager.diskBlockManager.getFile(ShuffleDataBlockId(shuffleId, mapId, NOOP_REDUCE_ID)) } - private def getIndexFile(shuffleId: Int, mapTaskId: Long): File = { - blockManager.diskBlockManager.getFile( - ShuffleIndexBlockId(shuffleId, mapTaskId, NOOP_REDUCE_ID)) + private def getIndexFile(shuffleId: Int, mapId: Long): File = { + blockManager.diskBlockManager.getFile(ShuffleIndexBlockId(shuffleId, mapId, NOOP_REDUCE_ID)) } /** * Remove data file and index file that contain the output data from one map. */ - def removeDataByMap(shuffleId: Int, mapTaskId: Long): Unit = { - var file = getDataFile(shuffleId, mapTaskId) + def removeDataByMap(shuffleId: Int, mapId: Long): Unit = { + var file = getDataFile(shuffleId, mapId) if (file.exists()) { if (!file.delete()) { logWarning(s"Error deleting data ${file.getPath()}") } } - file = getIndexFile(shuffleId, mapTaskId) + file = getIndexFile(shuffleId, mapId) if (file.exists()) { if (!file.delete()) { logWarning(s"Error deleting index ${file.getPath()}") @@ -136,13 +135,13 @@ private[spark] class IndexShuffleBlockResolver( */ def writeIndexFileAndCommit( shuffleId: Int, - mapTaskId: Long, + mapId: Long, lengths: Array[Long], dataTmp: File): Unit = { - val indexFile = getIndexFile(shuffleId, mapTaskId) + val indexFile = getIndexFile(shuffleId, mapId) val indexTmp = Utils.tempFileWith(indexFile) try { - val dataFile = getDataFile(shuffleId, mapTaskId) + val dataFile = getDataFile(shuffleId, mapId) // There is only one IndexShuffleBlockResolver per executor, this synchronization make sure // the following check and rename are atomic. synchronized { diff --git a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala index d0869519a2f74..e677b896ee6e5 100644 --- a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala @@ -591,8 +591,8 @@ final class ShuffleBlockFetcherIterator( address: BlockManagerId, e: Throwable) = { blockId match { - case ShuffleBlockId(shufId, mapTaskId, reduceId) => - throw new FetchFailedException(address, shufId, mapTaskId, mapIndex, reduceId, e) + case ShuffleBlockId(shufId, mapId, reduceId) => + throw new FetchFailedException(address, shufId, mapId, mapIndex, reduceId, e) case _ => throw new SparkException( "Failed to get block " + blockId + ", which is not a shuffle block", e) diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala index 45beb42f4be30..6fecfbaca8416 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala @@ -727,7 +727,7 @@ private[spark] class ExternalSorter[K, V, C]( */ def writePartitionedMapOutput( shuffleId: Int, - mapTaskId: Long, + mapId: Long, mapOutputWriter: ShuffleMapOutputWriter): Unit = { var nextPartitionId = 0 if (spills.isEmpty) { @@ -740,7 +740,7 @@ private[spark] class ExternalSorter[K, V, C]( var partitionPairsWriter: ShufflePartitionPairsWriter = null TryUtils.tryWithSafeFinally { partitionWriter = mapOutputWriter.getPartitionWriter(partitionId) - val blockId = ShuffleBlockId(shuffleId, mapTaskId, partitionId) + val blockId = ShuffleBlockId(shuffleId, mapId, partitionId) partitionPairsWriter = new ShufflePartitionPairsWriter( partitionWriter, serializerManager, @@ -760,7 +760,7 @@ private[spark] class ExternalSorter[K, V, C]( } else { // We must perform merge-sort; get an iterator by partition and write everything directly. for ((id, elements) <- this.partitionedIterator) { - val blockId = ShuffleBlockId(shuffleId, mapTaskId, id) + val blockId = ShuffleBlockId(shuffleId, mapId, id) var partitionWriter: ShufflePartitionWriter = null var partitionPairsWriter: ShufflePartitionPairsWriter = null TryUtils.tryWithSafeFinally { diff --git a/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderSuite.scala index 0cc1c9274fe61..d0cbb30fe0232 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderSuite.scala @@ -87,14 +87,14 @@ class BlockStoreShuffleReaderSuite extends SparkFunSuite with LocalSparkContext // Setup the mocked BlockManager to return RecordingManagedBuffers. val localBlockManagerId = BlockManagerId("test-client", "test-client", 1) when(blockManager.blockManagerId).thenReturn(localBlockManagerId) - val buffers = (0 until numMaps).map { mapTaskId => + val buffers = (0 until numMaps).map { mapId => // Create a ManagedBuffer with the shuffle data. val nioBuffer = new NioManagedBuffer(ByteBuffer.wrap(byteOutputStream.toByteArray)) val managedBuffer = new RecordingManagedBuffer(nioBuffer) // Setup the blockManager mock so the buffer gets returned when the shuffle code tries to // fetch shuffle data. - val shuffleBlockId = ShuffleBlockId(shuffleId, mapTaskId, reduceId) + val shuffleBlockId = ShuffleBlockId(shuffleId, mapId, reduceId) when(blockManager.getBlockData(shuffleBlockId)).thenReturn(managedBuffer) managedBuffer } @@ -103,12 +103,12 @@ class BlockStoreShuffleReaderSuite extends SparkFunSuite with LocalSparkContext // shuffle data to read. val mapOutputTracker = mock(classOf[MapOutputTracker]) when(mapOutputTracker.getMapSizesByExecutorId( - shuffleId, reduceId, reduceId + 1, false)).thenReturn { + shuffleId, reduceId, reduceId + 1, useOldFetchProtocol = false)).thenReturn { // Test a scenario where all data is local, to avoid creating a bunch of additional mocks // for the code to read data over the network. - val shuffleBlockIdsAndSizes = (0 until numMaps).map { mapTaskId => - val shuffleBlockId = ShuffleBlockId(shuffleId, mapTaskId, reduceId) - (shuffleBlockId, byteOutputStream.size().toLong, mapTaskId) + val shuffleBlockIdsAndSizes = (0 until numMaps).map { mapId => + val shuffleBlockId = ShuffleBlockId(shuffleId, mapId, reduceId) + (shuffleBlockId, byteOutputStream.size().toLong, mapId) } Seq((localBlockManagerId, shuffleBlockIdsAndSizes)).toIterator } diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala index 808c0360ee99d..f8474022867f4 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala @@ -138,7 +138,7 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte val writer = new BypassMergeSortShuffleWriter[Int, Int]( blockManager, shuffleHandle, - 0L, // MapTaskAttemptId + 0L, // MapId conf, taskContext.taskMetrics().shuffleWriteMetrics, shuffleExecutorComponents) @@ -164,7 +164,7 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte val writer = new BypassMergeSortShuffleWriter[Int, Int]( blockManager, shuffleHandle, - 0L, + 0L, // MapId transferConf, taskContext.taskMetrics().shuffleWriteMetrics, shuffleExecutorComponents) @@ -199,7 +199,7 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte val writer = new BypassMergeSortShuffleWriter[Int, Int]( blockManager, shuffleHandle, - 0L, + 0L, // MapId conf, taskContext.taskMetrics().shuffleWriteMetrics, shuffleExecutorComponents) @@ -220,7 +220,7 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte val writer = new BypassMergeSortShuffleWriter[Int, Int]( blockManager, shuffleHandle, - 0L, + 0L, // MapId conf, taskContext.taskMetrics().shuffleWriteMetrics, shuffleExecutorComponents) diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/IndexShuffleBlockResolverSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/IndexShuffleBlockResolverSuite.scala index b087c2fbd0c88..27bb06b4e0636 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/IndexShuffleBlockResolverSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/IndexShuffleBlockResolverSuite.scala @@ -60,8 +60,8 @@ class IndexShuffleBlockResolverSuite extends SparkFunSuite with BeforeAndAfterEa test("commit shuffle files multiple times") { val shuffleId = 1 - val mapTaskId = 2 - val idxName = s"shuffle_${shuffleId}_${mapTaskId}_0.index" + val mapId = 2 + val idxName = s"shuffle_${shuffleId}_${mapId}_0.index" val resolver = new IndexShuffleBlockResolver(conf, blockManager) val lengths = Array[Long](10, 0, 20) val dataTmp = File.createTempFile("shuffle", null, tempDir) @@ -71,10 +71,10 @@ class IndexShuffleBlockResolverSuite extends SparkFunSuite with BeforeAndAfterEa } { out.close() } - resolver.writeIndexFileAndCommit(shuffleId, mapTaskId, lengths, dataTmp) + resolver.writeIndexFileAndCommit(shuffleId, mapId, lengths, dataTmp) val indexFile = new File(tempDir.getAbsolutePath, idxName) - val dataFile = resolver.getDataFile(shuffleId, mapTaskId) + val dataFile = resolver.getDataFile(shuffleId, mapId) assert(indexFile.exists()) assert(indexFile.length() === (lengths.length + 1) * 8) @@ -91,7 +91,7 @@ class IndexShuffleBlockResolverSuite extends SparkFunSuite with BeforeAndAfterEa } { out2.close() } - resolver.writeIndexFileAndCommit(shuffleId, mapTaskId, lengths2, dataTmp2) + resolver.writeIndexFileAndCommit(shuffleId, mapId, lengths2, dataTmp2) assert(indexFile.length() === (lengths.length + 1) * 8) assert(lengths2.toSeq === lengths.toSeq) @@ -130,7 +130,7 @@ class IndexShuffleBlockResolverSuite extends SparkFunSuite with BeforeAndAfterEa } { out3.close() } - resolver.writeIndexFileAndCommit(shuffleId, mapTaskId, lengths3, dataTmp3) + resolver.writeIndexFileAndCommit(shuffleId, mapId, lengths3, dataTmp3) assert(indexFile.length() === (lengths3.length + 1) * 8) assert(lengths3.toSeq != lengths.toSeq) assert(dataFile.exists()) From d2215b2554708c535fdd3a42ab62c5559711bf27 Mon Sep 17 00:00:00 2001 From: Yuanjian Li Date: Thu, 19 Sep 2019 02:05:16 +0800 Subject: [PATCH 14/15] Keep mapId in shuffle writer and other comments --- .../api/ShuffleExecutorComponents.java | 1 + .../shuffle/sort/UnsafeShuffleWriter.java | 3 +- .../apache/spark/shuffle/ShuffleManager.scala | 1 + .../spark/shuffle/ShuffleWriteProcessor.scala | 1 + .../shuffle/sort/SortShuffleManager.scala | 6 ++-- .../shuffle/sort/SortShuffleWriter.scala | 8 ++--- .../storage/ShuffleBlockFetcherIterator.scala | 33 ++++++++++++------- .../sort/UnsafeShuffleWriterSuite.java | 2 ++ .../scala/org/apache/spark/ShuffleSuite.scala | 4 +-- .../spark/memory/MemoryTestingUtils.scala | 4 +-- .../shuffle/sort/SortShuffleWriterSuite.scala | 6 ++-- 11 files changed, 45 insertions(+), 24 deletions(-) diff --git a/core/src/main/java/org/apache/spark/shuffle/api/ShuffleExecutorComponents.java b/core/src/main/java/org/apache/spark/shuffle/api/ShuffleExecutorComponents.java index 042d0e5ae2b46..d30f3dad3c940 100644 --- a/core/src/main/java/org/apache/spark/shuffle/api/ShuffleExecutorComponents.java +++ b/core/src/main/java/org/apache/spark/shuffle/api/ShuffleExecutorComponents.java @@ -40,6 +40,7 @@ public interface ShuffleExecutorComponents { /** * Called once per map task to create a writer that will be responsible for persisting all the * partitioned bytes written by that map task. + * * @param shuffleId Unique identifier for the shuffle the map task is a part of * @param mapId An ID of the map task. The ID is unique within this Spark application. * @param numPartitions The number of partitions that will be written by the map task. Some of diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java index 2070369774cd5..d09282e61a9c7 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java @@ -109,6 +109,7 @@ public UnsafeShuffleWriter( BlockManager blockManager, TaskMemoryManager memoryManager, SerializedShuffleHandle handle, + long mapId, TaskContext taskContext, SparkConf sparkConf, ShuffleWriteMetricsReporter writeMetrics, @@ -122,7 +123,7 @@ public UnsafeShuffleWriter( } this.blockManager = blockManager; this.memoryManager = memoryManager; - this.mapId = taskContext.taskAttemptId(); + this.mapId = mapId; final ShuffleDependency dep = handle.dependency(); this.shuffleId = dep.shuffleId(); this.serializer = dep.serializer().newInstance(); diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala index a2fd2da6fe97a..a717ef242ea7c 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala @@ -39,6 +39,7 @@ private[spark] trait ShuffleManager { /** Get a writer for a given partition. Called on executors by map tasks. */ def getWriter[K, V]( handle: ShuffleHandle, + mapId: Long, context: TaskContext, metrics: ShuffleWriteMetricsReporter): ShuffleWriter[K, V] diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriteProcessor.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriteProcessor.scala index 68b589316e9e2..f222200a7816c 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriteProcessor.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriteProcessor.scala @@ -51,6 +51,7 @@ private[spark] class ShuffleWriteProcessor extends Serializable with Logging { val manager = SparkEnv.get.shuffleManager writer = manager.getWriter[Any, Any]( dep.shuffleHandle, + context.taskAttemptId(), context, createMetricsReporter(context)) writer.write( diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala index b397c36c72b34..d96bcb3d073df 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala @@ -130,6 +130,7 @@ private[spark] class SortShuffleManager(conf: SparkConf) extends ShuffleManager /** Get a writer for a given partition. Called on executors by map tasks. */ override def getWriter[K, V]( handle: ShuffleHandle, + mapId: Long, context: TaskContext, metrics: ShuffleWriteMetricsReporter): ShuffleWriter[K, V] = { val mapTaskIds = taskIdMapsForShuffle.computeIfAbsent( @@ -142,6 +143,7 @@ private[spark] class SortShuffleManager(conf: SparkConf) extends ShuffleManager env.blockManager, context.taskMemoryManager(), unsafeShuffleHandle, + mapId, context, env.conf, metrics, @@ -150,13 +152,13 @@ private[spark] class SortShuffleManager(conf: SparkConf) extends ShuffleManager new BypassMergeSortShuffleWriter( env.blockManager, bypassMergeSortHandle, - context.taskAttemptId(), + mapId, env.conf, metrics, shuffleExecutorComponents) case other: BaseShuffleHandle[K @unchecked, V @unchecked, _] => new SortShuffleWriter( - shuffleBlockResolver, other, context, shuffleExecutorComponents) + shuffleBlockResolver, other, mapId, context, shuffleExecutorComponents) } } diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala index 7e209f0b825fc..a391bdf2db44e 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala @@ -27,6 +27,7 @@ import org.apache.spark.util.collection.ExternalSorter private[spark] class SortShuffleWriter[K, V, C]( shuffleBlockResolver: IndexShuffleBlockResolver, handle: BaseShuffleHandle[K, V, C], + mapId: Long, context: TaskContext, shuffleExecutorComponents: ShuffleExecutorComponents) extends ShuffleWriter[K, V] with Logging { @@ -63,12 +64,11 @@ private[spark] class SortShuffleWriter[K, V, C]( // Don't bother including the time to open the merged output file in the shuffle write time, // because it just opens a single file, so is typically too fast to measure accurately // (see SPARK-3570). - val mapTaskId = context.taskAttemptId() val mapOutputWriter = shuffleExecutorComponents.createMapOutputWriter( - dep.shuffleId, mapTaskId, dep.partitioner.numPartitions) - sorter.writePartitionedMapOutput(dep.shuffleId, mapTaskId, mapOutputWriter) + dep.shuffleId, mapId, dep.partitioner.numPartitions) + sorter.writePartitionedMapOutput(dep.shuffleId, mapId, mapOutputWriter) val partitionLengths = mapOutputWriter.commitAllPartitions() - mapStatus = MapStatus(blockManager.shuffleServerId, partitionLengths, mapTaskId) + mapStatus = MapStatus(blockManager.shuffleServerId, partitionLengths, mapId) } /** Close this writer, passing along whether the map completed */ diff --git a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala index e677b896ee6e5..5fce358fae37f 100644 --- a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala @@ -227,10 +227,10 @@ final class ShuffleBlockFetcherIterator( // so we can look up the block info of each blockID val infoMap = req.blocks.map { - case (blockId, size, mapIndex) => (blockId.toString, (size, mapIndex)) + case FetchBlockInfo(blockId, size, mapIndex) => (blockId.toString, (size, mapIndex)) }.toMap val remainingBlocks = new HashSet[String]() ++= infoMap.keys - val blockIds = req.blocks.map(_._1.toString) + val blockIds = req.blocks.map(_.blockId.toString) val address = req.address val blockFetchingListener = new BlockFetchingListener { @@ -298,7 +298,7 @@ final class ShuffleBlockFetcherIterator( } else { val iterator = blockInfos.iterator var curRequestSize = 0L - var curBlocks = new ArrayBuffer[(BlockId, Long, Int)] + var curBlocks = new ArrayBuffer[FetchBlockInfo] while (iterator.hasNext) { val (blockId, size, mapIndex) = iterator.next() remoteBlockBytes += size @@ -307,7 +307,7 @@ final class ShuffleBlockFetcherIterator( } else if (size == 0) { throw new BlockException(blockId, "Zero-sized blocks should be excluded.") } else { - curBlocks += ((blockId, size, mapIndex)) + curBlocks += FetchBlockInfo(blockId, size, mapIndex) remoteBlocks += blockId numBlocksToFetch += 1 curRequestSize += size @@ -318,7 +318,7 @@ final class ShuffleBlockFetcherIterator( remoteRequests += new FetchRequest(address, curBlocks) logDebug(s"Creating fetch request of $curRequestSize at $address " + s"with ${curBlocks.size} blocks") - curBlocks = new ArrayBuffer[(BlockId, Long, Int)] + curBlocks = new ArrayBuffer[FetchBlockInfo] curRequestSize = 0 } } @@ -494,7 +494,8 @@ final class ShuffleBlockFetcherIterator( } else { logWarning(s"got an corrupted block $blockId from $address, fetch again", e) corruptedBlocks += blockId - fetchRequests += FetchRequest(address, Array((blockId, size, mapIndex))) + fetchRequests += FetchRequest( + address, Array(FetchBlockInfo(blockId, size, mapIndex))) result = null } } finally { @@ -696,15 +697,25 @@ private class ShuffleFetchCompletionListener(var data: ShuffleBlockFetcherIterat private[storage] object ShuffleBlockFetcherIterator { + /** + * The block information to fetch used in FetchRequest. + * @param blockId block id + * @param size estimated size of the block. Note that this is NOT the exact bytes. + * Size of remote block is used to calculate bytesInFlight. + * @param mapIndex the mapIndex for this block, which indicate the index in the map stage. + */ + private[storage] case class FetchBlockInfo( + blockId: BlockId, + size: Long, + mapIndex: Int) + /** * A request to fetch blocks from a remote BlockManager. * @param address remote BlockManager to fetch from. - * @param blocks Sequence of tuple, where the first element is the block id, - * and the second element is the estimated size, used to calculate bytesInFlight, - * the third element is the mapIndex. + * @param blocks Sequence of the information for blocks to fetch from the same address. */ - case class FetchRequest(address: BlockManagerId, blocks: Seq[(BlockId, Long, Int)]) { - val size = blocks.map(_._2).sum + case class FetchRequest(address: BlockManagerId, blocks: Seq[FetchBlockInfo]) { + val size = blocks.map(_.size).sum } /** diff --git a/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java b/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java index 244aa9a6baa9c..a901ae62e8cd8 100644 --- a/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java +++ b/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java @@ -178,6 +178,7 @@ private UnsafeShuffleWriter createWriter(boolean transferToEnabled) { blockManager, taskMemoryManager, new SerializedShuffleHandle<>(0, shuffleDep), + 0L, // map id taskContext, conf, taskContext.taskMetrics().shuffleWriteMetrics(), @@ -536,6 +537,7 @@ public void testPeakMemoryUsed() throws Exception { blockManager, taskMemoryManager, new SerializedShuffleHandle<>(0, shuffleDep), + 0L, // map id taskContext, conf, taskContext.taskMetrics().shuffleWriteMetrics(), diff --git a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala index 5dc62f78449fe..c75b56315547c 100644 --- a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala +++ b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala @@ -367,7 +367,7 @@ abstract class ShuffleSuite extends SparkFunSuite with Matchers with LocalSparkC val context1 = new TaskContextImpl(0, 0, 0, 0L, 0, taskMemoryManager, new Properties, metricsSystem) val writer1 = manager.getWriter[Int, Int]( - shuffleHandle, context1, context1.taskMetrics.shuffleWriteMetrics) + shuffleHandle, 0, context1, context1.taskMetrics.shuffleWriteMetrics) val data1 = (1 to 10).map { x => x -> x} // second attempt -- also successful. We'll write out different data, @@ -376,7 +376,7 @@ abstract class ShuffleSuite extends SparkFunSuite with Matchers with LocalSparkC val context2 = new TaskContextImpl(0, 0, 0, 1L, 0, taskMemoryManager, new Properties, metricsSystem) val writer2 = manager.getWriter[Int, Int]( - shuffleHandle, context2, context2.taskMetrics.shuffleWriteMetrics) + shuffleHandle, 0, context2, context2.taskMetrics.shuffleWriteMetrics) val data2 = (11 to 20).map { x => x -> x} // interleave writes of both attempts -- we want to test that both attempts can occur diff --git a/core/src/test/scala/org/apache/spark/memory/MemoryTestingUtils.scala b/core/src/test/scala/org/apache/spark/memory/MemoryTestingUtils.scala index 078392279fa05..dcf89e4f75acf 100644 --- a/core/src/test/scala/org/apache/spark/memory/MemoryTestingUtils.scala +++ b/core/src/test/scala/org/apache/spark/memory/MemoryTestingUtils.scala @@ -25,13 +25,13 @@ import org.apache.spark.{SparkEnv, TaskContext, TaskContextImpl} * Helper methods for mocking out memory-management-related classes in tests. */ object MemoryTestingUtils { - def fakeTaskContext(env: SparkEnv, mapTaskId: Long = 0): TaskContext = { + def fakeTaskContext(env: SparkEnv): TaskContext = { val taskMemoryManager = new TaskMemoryManager(env.memoryManager, 0) new TaskContextImpl( stageId = 0, stageAttemptNumber = 0, partitionId = 0, - taskAttemptId = mapTaskId, + taskAttemptId = 0, attemptNumber = 0, taskMemoryManager = taskMemoryManager, localProperties = new Properties, diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleWriterSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleWriterSuite.scala index 0923b5ffd5e59..4c5694fcf0305 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleWriterSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleWriterSuite.scala @@ -72,10 +72,11 @@ class SortShuffleWriterSuite extends SparkFunSuite with SharedSparkContext with } test("write empty iterator") { - val context = MemoryTestingUtils.fakeTaskContext(sc.env, mapTaskId = 1) + val context = MemoryTestingUtils.fakeTaskContext(sc.env) val writer = new SortShuffleWriter[Int, Int, Int]( shuffleBlockResolver, shuffleHandle, + mapId = 1, context, shuffleExecutorComponents) writer.write(Iterator.empty) @@ -88,11 +89,12 @@ class SortShuffleWriterSuite extends SparkFunSuite with SharedSparkContext with } test("write with some records") { - val context = MemoryTestingUtils.fakeTaskContext(sc.env, mapTaskId = 2) + val context = MemoryTestingUtils.fakeTaskContext(sc.env) val records = List[(Int, Int)]((1, 2), (2, 3), (4, 4), (6, 5)) val writer = new SortShuffleWriter[Int, Int, Int]( shuffleBlockResolver, shuffleHandle, + mapId = 2, context, shuffleExecutorComponents) writer.write(records.toIterator) From 28c9f9c2da1215a836fef7925e95a40c7c6dd87e Mon Sep 17 00:00:00 2001 From: Yuanjian Li Date: Thu, 19 Sep 2019 02:26:04 +0800 Subject: [PATCH 15/15] last comment --- .../apache/spark/scheduler/DAGSchedulerSuite.scala | 12 ++++++++---- 1 file changed, 8 insertions(+), 4 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index a1d51cf429857..f6c0bf61f6d9e 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -487,17 +487,21 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // map stage1 completes successfully, with one task on each executor complete(taskSets(0), Seq( (Success, - MapStatus(BlockManagerId("exec-hostA1", "hostA", 12345), Array.fill[Long](1)(2), 5)), + MapStatus( + BlockManagerId("exec-hostA1", "hostA", 12345), Array.fill[Long](1)(2), mapTaskId = 5)), (Success, - MapStatus(BlockManagerId("exec-hostA2", "hostA", 12345), Array.fill[Long](1)(2), 6)), + MapStatus( + BlockManagerId("exec-hostA2", "hostA", 12345), Array.fill[Long](1)(2), mapTaskId = 6)), (Success, makeMapStatus("hostB", 1, mapTaskId = 7)) )) // map stage2 completes successfully, with one task on each executor complete(taskSets(1), Seq( (Success, - MapStatus(BlockManagerId("exec-hostA1", "hostA", 12345), Array.fill[Long](1)(2), 8)), + MapStatus( + BlockManagerId("exec-hostA1", "hostA", 12345), Array.fill[Long](1)(2), mapTaskId = 8)), (Success, - MapStatus(BlockManagerId("exec-hostA2", "hostA", 12345), Array.fill[Long](1)(2), 9)), + MapStatus( + BlockManagerId("exec-hostA2", "hostA", 12345), Array.fill[Long](1)(2), mapTaskId = 9)), (Success, makeMapStatus("hostB", 1, mapTaskId = 10)) )) // make sure our test setup is correct