From 31cdd239c1367da8750ce512eab82d4684f6559b Mon Sep 17 00:00:00 2001 From: Yuanjian Li Date: Mon, 19 Nov 2018 11:06:12 +0800 Subject: [PATCH 01/16] Changes on reader writer and extend shuffle block id --- .../shuffle/ExternalShuffleBlockHandler.java | 26 +++-- .../shuffle/ExternalShuffleBlockResolver.java | 30 ++++-- .../ExternalShuffleBlockHandlerSuite.java | 8 +- .../sort/BypassMergeSortShuffleWriter.java | 10 +- .../shuffle/sort/UnsafeShuffleWriter.java | 8 +- .../org/apache/spark/BarrierTaskContext.scala | 7 ++ .../org/apache/spark/MapOutputTracker.scala | 97 +++++++++++++++++-- .../scala/org/apache/spark/TaskContext.scala | 11 +++ .../org/apache/spark/TaskContextImpl.scala | 12 ++- .../apache/spark/scheduler/DAGScheduler.scala | 15 +-- .../spark/scheduler/ShuffleMapTask.scala | 5 +- .../org/apache/spark/scheduler/Stage.scala | 9 ++ .../org/apache/spark/scheduler/Task.scala | 6 +- .../shuffle/BlockStoreShuffleReader.scala | 7 +- .../shuffle/IndexShuffleBlockResolver.scala | 42 +++++--- .../shuffle/sort/SortShuffleManager.scala | 18 ++-- .../shuffle/sort/SortShuffleWriter.scala | 13 ++- .../org/apache/spark/storage/BlockId.scala | 52 ++++++++-- .../storage/ShuffleBlockFetcherIterator.scala | 2 +- .../sort/UnsafeShuffleWriterSuite.java | 6 +- .../apache/spark/ContextCleanerSuite.scala | 4 +- .../BlockStoreShuffleReaderSuite.scala | 4 + .../BypassMergeSortShuffleWriterSuite.scala | 6 +- .../apache/spark/storage/BlockIdSuite.scala | 42 ++++++++ 24 files changed, 362 insertions(+), 78 deletions(-) diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java index 70dcc8b8b8b6..95737e8dcb62 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java @@ -218,21 +218,20 @@ private class ManagedBufferIterator implements Iterator { private final int shuffleId; // An array containing mapId and reduceId pairs. private final int[] mapIdAndReduceIds; + private final int indeterminateRetryId; ManagedBufferIterator(String appId, String execId, String[] blockIds) { this.appId = appId; this.execId = execId; String[] blockId0Parts = blockIds[0].split("_"); - if (blockId0Parts.length != 4 || !blockId0Parts[0].equals("shuffle")) { - throw new IllegalArgumentException("Unexpected shuffle block id format: " + blockIds[0]); - } + checkBlockId(blockId0Parts, blockIds, 0); this.shuffleId = Integer.parseInt(blockId0Parts[1]); mapIdAndReduceIds = new int[2 * blockIds.length]; + this.indeterminateRetryId = + (blockId0Parts.length == 5) ? Integer.parseInt(blockId0Parts[4]) : -1; for (int i = 0; i < blockIds.length; i++) { String[] blockIdParts = blockIds[i].split("_"); - if (blockIdParts.length != 4 || !blockIdParts[0].equals("shuffle")) { - throw new IllegalArgumentException("Unexpected shuffle block id format: " + blockIds[i]); - } + checkBlockId(blockIdParts, blockIds, i); if (Integer.parseInt(blockIdParts[1]) != shuffleId) { throw new IllegalArgumentException("Expected shuffleId=" + shuffleId + ", got:" + blockIds[i]); @@ -242,6 +241,19 @@ private class ManagedBufferIterator implements Iterator { } } + private void checkBlockId(String[] blockIdParts, String[] wholeBlockId, int index) { + if ((blockIdParts.length != 4 && blockIdParts.length != 5) + || !blockIdParts[0].equals("shuffle")) { + throw new IllegalArgumentException( + "Unexpected shuffle block id format: " + wholeBlockId[index]); + } + if (blockIdParts.length == 5 && blockIdParts[4].equals("0")) { + throw new IllegalArgumentException( + "Unexpected indeterminateRetryId in shuffle block id, if indeterminateRetryId be set," + + " it should only up to 0: " + wholeBlockId[index]); + } + } + @Override public boolean hasNext() { return index < mapIdAndReduceIds.length; @@ -250,7 +262,7 @@ public boolean hasNext() { @Override public ManagedBuffer next() { final ManagedBuffer block = blockManager.getBlockData(appId, execId, shuffleId, - mapIdAndReduceIds[index], mapIdAndReduceIds[index + 1]); + mapIdAndReduceIds[index], mapIdAndReduceIds[index + 1], indeterminateRetryId); index += 2; metrics.blockTransferRateBytes.mark(block != null ? block.size() : 0); return block; 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 0b7a27402369..9ef2044fb03f 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 @@ -161,6 +161,18 @@ public void registerExecutor( executors.put(fullId, executorInfo); } + /** + * Overload getBlockData with setting indeterminateRetryId to an invalid value of -1. + */ + public ManagedBuffer getBlockData( + String appId, + String execId, + int shuffleId, + int mapId, + int reduceId) { + return getBlockData(appId, execId, shuffleId, mapId, reduceId, -1); + } + /** * Obtains a FileSegmentManagedBuffer from (shuffleId, mapId, reduceId). We make assumptions * about how the hash and sort based shuffles store their data. @@ -170,13 +182,15 @@ public ManagedBuffer getBlockData( String execId, int shuffleId, int mapId, - int reduceId) { + int reduceId, + int indeterminateRetryId) { 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, mapId, reduceId, indeterminateRetryId); } /** @@ -280,17 +294,21 @@ public boolean accept(File dir, String name) { * 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, + int mapId, int reduceId, int indeterminateRetryId) { + String baseFileName = "shuffle_" + shuffleId + "_" + mapId + "_0"; + if (indeterminateRetryId != -1) { + baseFileName = baseFileName + "_" + indeterminateRetryId; + } File indexFile = getFile(executor.localDirs, executor.subDirsPerLocalDir, - "shuffle_" + shuffleId + "_" + mapId + "_0.index"); + baseFileName + ".index"); try { ShuffleIndexInformation shuffleIndexInformation = shuffleIndexCache.get(indexFile); ShuffleIndexRecord shuffleIndexRecord = shuffleIndexInformation.getIndex(reduceId); return new FileSegmentManagedBuffer( conf, - getFile(executor.localDirs, executor.subDirsPerLocalDir, - "shuffle_" + shuffleId + "_" + mapId + "_0.data"), + getFile(executor.localDirs, executor.subDirsPerLocalDir, baseFileName + ".data"), shuffleIndexRecord.getOffset(), shuffleIndexRecord.getLength()); } catch (ExecutionException e) { diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandlerSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandlerSuite.java index 537c277cd26b..63344aed63f8 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandlerSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandlerSuite.java @@ -85,8 +85,8 @@ public void testOpenShuffleBlocks() { ManagedBuffer block0Marker = new NioManagedBuffer(ByteBuffer.wrap(new byte[3])); ManagedBuffer block1Marker = new NioManagedBuffer(ByteBuffer.wrap(new byte[7])); - when(blockResolver.getBlockData("app0", "exec1", 0, 0, 0)).thenReturn(block0Marker); - when(blockResolver.getBlockData("app0", "exec1", 0, 0, 1)).thenReturn(block1Marker); + when(blockResolver.getBlockData("app0", "exec1", 0, 0, 0, -1)).thenReturn(block0Marker); + when(blockResolver.getBlockData("app0", "exec1", 0, 0, 1, -1)).thenReturn(block1Marker); ByteBuffer openBlocks = new OpenBlocks("app0", "exec1", new String[] { "shuffle_0_0_0", "shuffle_0_0_1" }) .toByteBuffer(); @@ -109,8 +109,8 @@ public void testOpenShuffleBlocks() { assertEquals(block0Marker, buffers.next()); assertEquals(block1Marker, buffers.next()); assertFalse(buffers.hasNext()); - verify(blockResolver, times(1)).getBlockData("app0", "exec1", 0, 0, 0); - verify(blockResolver, times(1)).getBlockData("app0", "exec1", 0, 0, 1); + verify(blockResolver, times(1)).getBlockData("app0", "exec1", 0, 0, 0, -1); + verify(blockResolver, times(1)).getBlockData("app0", "exec1", 0, 0, 1, -1); // Verify open block request latency metrics Timer openBlockRequestLatencyMillis = (Timer) ((ExternalShuffleBlockHandler) handler) 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 32b446785a9f..9d35963e14b0 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 @@ -83,6 +83,7 @@ final class BypassMergeSortShuffleWriter extends ShuffleWriter { private final int mapId; private final Serializer serializer; private final IndexShuffleBlockResolver shuffleBlockResolver; + private final Option indeterminateAttemptId; /** Array of file writers, one for each partition */ private DiskBlockObjectWriter[] partitionWriters; @@ -116,6 +117,7 @@ final class BypassMergeSortShuffleWriter extends ShuffleWriter { this.writeMetrics = writeMetrics; this.serializer = dep.serializer(); this.shuffleBlockResolver = shuffleBlockResolver; + this.indeterminateAttemptId = taskContext.getIndeterminateAttemptId(); } @Override @@ -123,7 +125,8 @@ public void write(Iterator> records) throws IOException { assert (partitionWriters == null); if (!records.hasNext()) { partitionLengths = new long[numPartitions]; - shuffleBlockResolver.writeIndexFileAndCommit(shuffleId, mapId, partitionLengths, null); + shuffleBlockResolver.writeIndexFileAndCommit( + shuffleId, mapId, partitionLengths, null, indeterminateAttemptId); mapStatus = MapStatus$.MODULE$.apply(blockManager.shuffleServerId(), partitionLengths); return; } @@ -156,11 +159,12 @@ public void write(Iterator> records) throws IOException { } } - File output = shuffleBlockResolver.getDataFile(shuffleId, mapId); + File output = shuffleBlockResolver.getDataFile(shuffleId, mapId, indeterminateAttemptId); File tmp = Utils.tempFileWith(output); try { partitionLengths = writePartitionedFile(tmp); - shuffleBlockResolver.writeIndexFileAndCommit(shuffleId, mapId, partitionLengths, tmp); + shuffleBlockResolver.writeIndexFileAndCommit( + shuffleId, mapId, partitionLengths, tmp, indeterminateAttemptId); } finally { if (tmp.exists() && !tmp.delete()) { logger.error("Error while deleting temp file {}", tmp.getAbsolutePath()); 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 9d05f03613ce..486c62c1a2c3 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 @@ -82,6 +82,7 @@ public class UnsafeShuffleWriter extends ShuffleWriter { private final int initialSortBufferSize; private final int inputBufferSizeInBytes; private final int outputBufferSizeInBytes; + private final Option indeterminateAttemptId; @Nullable private MapStatus mapStatus; @Nullable private ShuffleExternalSorter sorter; @@ -150,6 +151,7 @@ public UnsafeShuffleWriter( this.outputBufferSizeInBytes = (int) (long) sparkConf.get(package$.MODULE$.SHUFFLE_UNSAFE_FILE_OUTPUT_BUFFER_SIZE()) * 1024; open(); + this.indeterminateAttemptId = taskContext.getIndeterminateAttemptId(); } private void updatePeakMemoryUsed() { @@ -231,7 +233,8 @@ void closeAndWriteOutput() throws IOException { final SpillInfo[] spills = sorter.closeAndGetSpills(); sorter = null; final long[] partitionLengths; - final File output = shuffleBlockResolver.getDataFile(shuffleId, mapId); + final File output = shuffleBlockResolver.getDataFile( + shuffleId, mapId, indeterminateAttemptId); final File tmp = Utils.tempFileWith(output); try { try { @@ -243,7 +246,8 @@ void closeAndWriteOutput() throws IOException { } } } - shuffleBlockResolver.writeIndexFileAndCommit(shuffleId, mapId, partitionLengths, tmp); + shuffleBlockResolver.writeIndexFileAndCommit( + shuffleId, mapId, partitionLengths, tmp, indeterminateAttemptId); } finally { if (tmp.exists() && !tmp.delete()) { logger.error("Error while deleting temp file {}", tmp.getAbsolutePath()); diff --git a/core/src/main/scala/org/apache/spark/BarrierTaskContext.scala b/core/src/main/scala/org/apache/spark/BarrierTaskContext.scala index a354f44a1be1..f3abc5239f7e 100644 --- a/core/src/main/scala/org/apache/spark/BarrierTaskContext.scala +++ b/core/src/main/scala/org/apache/spark/BarrierTaskContext.scala @@ -213,11 +213,18 @@ class BarrierTaskContext private[spark] ( taskContext.markTaskCompleted(error) } + override private[spark] def markAsIndeterminate(): Unit = { + taskContext.markAsIndeterminate() + } + override private[spark] def fetchFailed: Option[FetchFailedException] = { taskContext.fetchFailed } override private[spark] def getLocalProperties: Properties = taskContext.getLocalProperties + + override private[spark] def getIndeterminateAttemptId: Option[Int] = + taskContext.getIndeterminateAttemptId } @Experimental diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 1d4b1ef9c9a1..a6c3c92d0cdd 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -84,6 +84,13 @@ private class ShuffleStatus(numPartitions: Int) { */ private[this] var _numAvailableOutputs: Int = 0 + /** + * The bookkeeping for the indeterminate attempt id for corresponding ShuffleMapStage, most time + * its value is None cause the ShuffleMapStage's [[org.apache.spark.rdd.DeterministicLevel]] is + * DETERMINATE. + */ + private[this] var _indeterminateAttemptId: Option[Int] = None + /** * Register a map output. If there is already a registered location for the map output then it * will be replaced by the new location. @@ -207,12 +214,36 @@ private class ShuffleStatus(numPartitions: Int) { } cachedSerializedMapStatus = null } + + /** + * Set the _indeterminateAttemptId when the first task for rerunning indeterminate stage + * successfully ended, check the value with all following tasks cause the indeterminate stage + * should whole stage rerun. + */ + def setAndCheckIndeterminateAttemptId(stageAttemptId: Int): Unit = synchronized { + if (_indeterminateAttemptId.isEmpty) { + _indeterminateAttemptId = Some(stageAttemptId) + } else { + require(_indeterminateAttemptId.get == stageAttemptId, "The indeterminate stage should" + + s" whole stage rerun by attempt id: ${_indeterminateAttemptId.get}, but found a task" + + s" with different stageAttemptId $stageAttemptId.") + } + } + + /** + * The indeterminate attempt id for current shuffle status. + */ + def indeterminateAttemptId: Option[Int] = synchronized { + _indeterminateAttemptId + } } private[spark] sealed trait MapOutputTrackerMessage private[spark] case class GetMapOutputStatuses(shuffleId: Int) extends MapOutputTrackerMessage private[spark] case object StopMapOutputTracker extends MapOutputTrackerMessage +private[spark] case class GetIndeterminateAttemptId(shuffleId: Int) + extends MapOutputTrackerMessage private[spark] case class GetMapOutputMessage(shuffleId: Int, context: RpcCallContext) @@ -233,6 +264,11 @@ private[spark] class MapOutputTrackerMasterEndpoint( logInfo("MapOutputTrackerMasterEndpoint stopped!") context.reply(true) stop() + + case GetIndeterminateAttemptId(shuffleId: Int) => + val hostPort = context.senderAddress.hostPort + logInfo(s"Asked to send indeterminate attempt id for shuffle $shuffleId to $hostPort") + context.reply(tracker.shuffleStatuses(shuffleId).indeterminateAttemptId) } } @@ -295,8 +331,12 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging * 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. */ - def getMapSizesByExecutorId(shuffleId: Int, startPartition: Int, endPartition: Int) - : Iterator[(BlockManagerId, Seq[(BlockId, Long)])] + def getMapSizesByExecutorId( + shuffleId: Int, + startPartition: Int, + endPartition: Int, + needFetchIndeterminateAttemptId: Boolean = false) + : Iterator[(BlockManagerId, Seq[(BlockId, Long)])] /** * Deletes map output status information for the specified shuffle stage. @@ -645,19 +685,33 @@ 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)])] = { + def getMapSizesByExecutorId( + shuffleId: Int, + startPartition: Int, + endPartition: Int, + needFetchIndeterminateAttemptId: Boolean = false) + : Iterator[(BlockManagerId, Seq[(BlockId, Long)])] = { logDebug(s"Fetching outputs for shuffle $shuffleId, partitions $startPartition-$endPartition") shuffleStatuses.get(shuffleId) match { case Some (shuffleStatus) => + val indeterminateAttemptId = if (needFetchIndeterminateAttemptId) { + shuffleStatuses(shuffleId).indeterminateAttemptId + } else { + None + } shuffleStatus.withMapStatuses { statuses => - MapOutputTracker.convertMapStatuses(shuffleId, startPartition, endPartition, statuses) + MapOutputTracker.convertMapStatuses( + shuffleId, startPartition, endPartition, statuses, indeterminateAttemptId) } case None => Iterator.empty } } + def registerIndeterminateShuffle(shuffleId: Int, stageAttemptId: Int): Unit = { + shuffleStatuses(shuffleId).setAndCheckIndeterminateAttemptId(stageAttemptId) + } + override def stop() { mapOutputRequests.offer(PoisonPill) threadpool.shutdown() @@ -678,16 +732,29 @@ private[spark] class MapOutputTrackerWorker(conf: SparkConf) extends MapOutputTr val mapStatuses: Map[Int, Array[MapStatus]] = new ConcurrentHashMap[Int, Array[MapStatus]]().asScala + val indeterminateAttemptIds: Map[Int, Option[Int]] = + new ConcurrentHashMap[Int, Option[Int]]().asScala + /** Remembers which map output locations are currently being fetched on an executor. */ private val fetching = new HashSet[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) + override def getMapSizesByExecutorId( + shuffleId: Int, + startPartition: Int, + endPartition: Int, + needFetchIndeterminateAttemptId: Boolean = false) : Iterator[(BlockManagerId, Seq[(BlockId, Long)])] = { logDebug(s"Fetching outputs for shuffle $shuffleId, partitions $startPartition-$endPartition") val statuses = getStatuses(shuffleId) + val indeterminateAttemptId = if (needFetchIndeterminateAttemptId) { + getIndeterminateAttemptId(shuffleId) + } else { + None + } try { - MapOutputTracker.convertMapStatuses(shuffleId, startPartition, endPartition, statuses) + MapOutputTracker.convertMapStatuses( + shuffleId, startPartition, endPartition, statuses, indeterminateAttemptId) } catch { case e: MetadataFetchFailedException => // We experienced a fetch failure so our mapStatuses cache is outdated; clear it: @@ -696,6 +763,17 @@ private[spark] class MapOutputTrackerWorker(conf: SparkConf) extends MapOutputTr } } + private def getIndeterminateAttemptId(shuffleId: Int): Option[Int] = { + val id = indeterminateAttemptIds.get(shuffleId).orNull + if (id == null) { + val fetchedId = askTracker[Option[Int]](GetIndeterminateAttemptId(shuffleId)) + indeterminateAttemptIds.put(shuffleId, fetchedId) + fetchedId + } else { + id + } + } + /** * Get or fetch the array of MapStatuses for a given shuffle ID. NOTE: clients MUST synchronize * on this array when reading it, because on the driver, we may be changing it in place. @@ -871,7 +949,8 @@ private[spark] object MapOutputTracker extends Logging { shuffleId: Int, startPartition: Int, endPartition: Int, - statuses: Array[MapStatus]): Iterator[(BlockManagerId, Seq[(BlockId, Long)])] = { + statuses: Array[MapStatus], + indeterminateAttemptId: Option[Int]): Iterator[(BlockManagerId, Seq[(BlockId, Long)])] = { assert (statuses != null) val splitsByAddress = new HashMap[BlockManagerId, ListBuffer[(BlockId, Long)]] for ((status, mapId) <- statuses.iterator.zipWithIndex) { @@ -884,7 +963,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, mapId, part, indeterminateAttemptId), size)) } } } diff --git a/core/src/main/scala/org/apache/spark/TaskContext.scala b/core/src/main/scala/org/apache/spark/TaskContext.scala index 959f246f3f9f..919fac2836a0 100644 --- a/core/src/main/scala/org/apache/spark/TaskContext.scala +++ b/core/src/main/scala/org/apache/spark/TaskContext.scala @@ -223,9 +223,20 @@ abstract class TaskContext extends Serializable { /** Marks the task as completed and triggers the completion listeners. */ private[spark] def markTaskCompleted(error: Option[Throwable]): Unit + /** Marks the task as indeterminate task. */ + private[spark] def markAsIndeterminate(): Unit + /** Optionally returns the stored fetch failure in the task. */ private[spark] def fetchFailed: Option[FetchFailedException] /** Gets local properties set upstream in the driver. */ private[spark] def getLocalProperties: Properties + + /** + * Get the indeterminate stage attempt id, it will return None in following two case: + * 1. The task is determinate. + * 2. The task is indeterminate but no retry occurred. + * Otherwise it return the stage attempt number. + */ + private[spark] def getIndeterminateAttemptId: Option[Int] } diff --git a/core/src/main/scala/org/apache/spark/TaskContextImpl.scala b/core/src/main/scala/org/apache/spark/TaskContextImpl.scala index 76296c5d0abd..39efb42c662c 100644 --- a/core/src/main/scala/org/apache/spark/TaskContextImpl.scala +++ b/core/src/main/scala/org/apache/spark/TaskContextImpl.scala @@ -73,6 +73,9 @@ private[spark] class TaskContextImpl( // Throwable that caused the task to fail private var failure: Throwable = _ + // Whether the task is determinate. + private var determinate: Boolean = true + // If there was a fetch failure in the task, we store it here, to make sure user-code doesn't // hide the exception. See SPARK-19276 @volatile private var _fetchFailedException: Option[FetchFailedException] = None @@ -143,6 +146,10 @@ private[spark] class TaskContextImpl( reasonIfKilled = Some(reason) } + private[spark] override def markAsIndeterminate(): Unit = { + determinate = false + } + private[spark] override def killTaskIfInterrupted(): Unit = { val reason = reasonIfKilled if (reason.isDefined) { @@ -174,5 +181,8 @@ private[spark] class TaskContextImpl( private[spark] override def fetchFailed: Option[FetchFailedException] = _fetchFailedException - private[spark] override def getLocalProperties(): Properties = localProperties + private[spark] override def getLocalProperties: Properties = localProperties + + private[spark] override def getIndeterminateAttemptId: Option[Int] = + if (!determinate) Some(stageAttemptNumber) else None } 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 524b0c4f6c3a..69ba5886a778 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1205,7 +1205,8 @@ private[spark] class DAGScheduler( stage.pendingPartitions += id new ShuffleMapTask(stage.id, stage.latestInfo.attemptNumber, taskBinary, part, locs, properties, serializedTaskMetrics, Option(jobId), - Option(sc.applicationId), sc.applicationAttemptId, stage.rdd.isBarrier()) + Option(sc.applicationId), sc.applicationAttemptId, stage.rdd.isBarrier(), + stage.isDeterminate()) } case stage: ResultStage => @@ -1547,7 +1548,6 @@ private[spark] class DAGScheduler( } abortStage(failedStage, abortMessage, None) } else { // update failedStages and make sure a ResubmitFailedStages event is enqueued - // TODO: Cancel running tasks in the failed stage -- cf. SPARK-17064 val noResubmitEnqueued = !failedStages.contains(failedStage) failedStages += failedStage failedStages += mapStage @@ -1591,11 +1591,12 @@ 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) + // Mark all the map as broken in the map stage, to ensure retry all the + // tasks on resubmitted stage attempt. Also mark this map stage as + // indeterminate, will rerun all its task and generate new intermediate + // shuffle file. + mapOutputTracker.unregisterAllMapOutput(mapStage.shuffleDep.shuffleId) + mapStage.markAsInDeterminate() } case resultStage: ResultStage if resultStage.activeJob.isDefined => 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 189e35ee8311..59159b54beee 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala @@ -63,9 +63,10 @@ private[spark] class ShuffleMapTask( jobId: Option[Int] = None, appId: Option[String] = None, appAttemptId: Option[String] = None, - isBarrier: Boolean = false) + isBarrier: Boolean = false, + isDeterminate: Boolean = true) extends Task[MapStatus](stageId, stageAttemptId, partition.index, localProperties, - serializedTaskMetrics, jobId, appId, appAttemptId, isBarrier) + serializedTaskMetrics, jobId, appId, appAttemptId, isBarrier, isDeterminate) with Logging { /** A constructor used only in test suites. This does not require passing in an RDD. */ 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 26cca334d3bd..1017a91d82c8 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala @@ -89,6 +89,9 @@ private[scheduler] abstract class Stage( */ val failedAttemptIds = new HashSet[Int] + /** The identification of this stage is a determinate stage and being retried. */ + private var determinate: Boolean = true + private[scheduler] def clearFailures() : Unit = { failedAttemptIds.clear() } @@ -116,4 +119,10 @@ 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 markAsInDeterminate(): Unit = { + determinate = false + } + + def isDeterminate(): Boolean = determinate } diff --git a/core/src/main/scala/org/apache/spark/scheduler/Task.scala b/core/src/main/scala/org/apache/spark/scheduler/Task.scala index c17a9038ba14..9e9b3cb7d37a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Task.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Task.scala @@ -51,6 +51,7 @@ import org.apache.spark.util._ * @param appAttemptId attempt id of the app this task belongs to * @param isBarrier whether this task belongs to a barrier stage. Spark must launch all the tasks * at the same time for a barrier stage. + * @param isDeterminate whether this task is determinate. */ private[spark] abstract class Task[T]( val stageId: Int, @@ -63,7 +64,8 @@ private[spark] abstract class Task[T]( val jobId: Option[Int] = None, val appId: Option[String] = None, val appAttemptId: Option[String] = None, - val isBarrier: Boolean = false) extends Serializable { + val isBarrier: Boolean = false, + val isDeterminate: Boolean = true) extends Serializable { @transient lazy val metrics: TaskMetrics = SparkEnv.get.closureSerializer.newInstance().deserialize(ByteBuffer.wrap(serializedTaskMetrics)) @@ -99,6 +101,8 @@ private[spark] abstract class Task[T]( taskContext } + if (!isDeterminate) context.markAsIndeterminate() + TaskContext.setTaskContext(context) taskThread = Thread.currentThread() 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 c7843710413d..d3029cd86ced 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala @@ -19,6 +19,7 @@ package org.apache.spark.shuffle import org.apache.spark._ import org.apache.spark.internal.{config, Logging} +import org.apache.spark.rdd.DeterministicLevel import org.apache.spark.serializer.SerializerManager import org.apache.spark.storage.{BlockManager, ShuffleBlockFetcherIterator} import org.apache.spark.util.CompletionIterator @@ -41,13 +42,17 @@ private[spark] class BlockStoreShuffleReader[K, C]( private val dep = handle.dependency + private val mapSideIndeterminate = + dep.rdd.outputDeterministicLevel == DeterministicLevel.INDETERMINATE + /** Read the combined key-values for this reduce task */ override def read(): Iterator[Product2[K, C]] = { val wrappedStreams = new ShuffleBlockFetcherIterator( context, blockManager.shuffleClient, blockManager, - mapOutputTracker.getMapSizesByExecutorId(handle.shuffleId, startPartition, endPartition), + mapOutputTracker.getMapSizesByExecutorId( + handle.shuffleId, startPartition, endPartition, mapSideIndeterminate), 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/IndexShuffleBlockResolver.scala b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala index d3f1c7ec1bbe..49dd69a0d908 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,43 @@ private[spark] class IndexShuffleBlockResolver( private val transportConf = SparkTransportConf.fromSparkConf(conf, "shuffle") - def getDataFile(shuffleId: Int, mapId: Int): File = { - blockManager.diskBlockManager.getFile(ShuffleDataBlockId(shuffleId, mapId, NOOP_REDUCE_ID)) + private def checkStageAttemptId(stageAttemptId: Option[Int]): Unit = { + require(stageAttemptId.isEmpty || (stageAttemptId.isDefined && stageAttemptId.get > 0), + "Setting stage attempt id here represent for rerunning indeterminate stage/task," + + "so it must > 0, otherwise it kept None as default.") } - private def getIndexFile(shuffleId: Int, mapId: Int): File = { - blockManager.diskBlockManager.getFile(ShuffleIndexBlockId(shuffleId, mapId, NOOP_REDUCE_ID)) + def getDataFile( + shuffleId: Int, + mapId: Int, + stageAttemptId: Option[Int] = None): File = { + checkStageAttemptId(stageAttemptId) + blockManager.diskBlockManager.getFile( + ShuffleDataBlockId(shuffleId, mapId, NOOP_REDUCE_ID, stageAttemptId)) + } + + private def getIndexFile( + shuffleId: Int, + mapId: Int, + stageAttemptId: Option[Int] = None): File = { + checkStageAttemptId(stageAttemptId) + blockManager.diskBlockManager.getFile( + ShuffleIndexBlockId(shuffleId, mapId, NOOP_REDUCE_ID, stageAttemptId)) } /** * Remove data file and index file that contain the output data from one map. */ - def removeDataByMap(shuffleId: Int, mapId: Int): Unit = { - var file = getDataFile(shuffleId, mapId) + def removeDataByMap( + shuffleId: Int, mapId: Int, indeterminateStageAttemptId: Option[Int]): Unit = { + var file = getDataFile(shuffleId, mapId, indeterminateStageAttemptId) if (file.exists()) { if (!file.delete()) { logWarning(s"Error deleting data ${file.getPath()}") } } - file = getIndexFile(shuffleId, mapId) + file = getIndexFile(shuffleId, mapId, indeterminateStageAttemptId) if (file.exists()) { if (!file.delete()) { logWarning(s"Error deleting index ${file.getPath()}") @@ -137,11 +154,12 @@ private[spark] class IndexShuffleBlockResolver( shuffleId: Int, mapId: Int, lengths: Array[Long], - dataTmp: File): Unit = { - val indexFile = getIndexFile(shuffleId, mapId) + dataTmp: File, + indeterminateAttemptId: Option[Int] = None): Unit = { + val indexFile = getIndexFile(shuffleId, mapId, indeterminateAttemptId) val indexTmp = Utils.tempFileWith(indexFile) try { - val dataFile = getDataFile(shuffleId, mapId) + val dataFile = getDataFile(shuffleId, mapId, indeterminateAttemptId) // There is only one IndexShuffleBlockResolver per executor, this synchronization make sure // the following check and rename are atomic. synchronized { @@ -193,7 +211,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.mapId, blockId.indeterminateAttemptId) // 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 +233,7 @@ private[spark] class IndexShuffleBlockResolver( } new FileSegmentManagedBuffer( transportConf, - getDataFile(blockId.shuffleId, blockId.mapId), + getDataFile(blockId.shuffleId, blockId.mapId, blockId.indeterminateAttemptId), 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 b59fa8e8a3cc..c8258aceec85 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 @@ -75,9 +75,10 @@ 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 tuple of number of mappers producing output and + * indeterminate stage attempt id for those shuffles. */ - private[this] val numMapsForShuffle = new ConcurrentHashMap[Int, Int]() + private[this] val numMapsForShuffle = new ConcurrentHashMap[Int, (Int, Option[Int])]() override val shuffleBlockResolver = new IndexShuffleBlockResolver(conf) @@ -128,7 +129,9 @@ private[spark] class SortShuffleManager(conf: SparkConf) extends ShuffleManager context: TaskContext, metrics: ShuffleWriteMetricsReporter): ShuffleWriter[K, V] = { numMapsForShuffle.putIfAbsent( - handle.shuffleId, handle.asInstanceOf[BaseShuffleHandle[_, _, _]].numMaps) + handle.shuffleId, + (handle.asInstanceOf[BaseShuffleHandle[_, _, _]].numMaps, + context.getIndeterminateAttemptId)) val env = SparkEnv.get handle match { case unsafeShuffleHandle: SerializedShuffleHandle[K @unchecked, V @unchecked] => @@ -156,10 +159,11 @@ private[spark] class SortShuffleManager(conf: SparkConf) extends ShuffleManager /** 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 => - shuffleBlockResolver.removeDataByMap(shuffleId, mapId) - } + Option(numMapsForShuffle.remove(shuffleId)).foreach { + case (numMaps, indeterminateAttemptId) => + (0 until numMaps).foreach { mapId => + shuffleBlockResolver.removeDataByMap(shuffleId, mapId, indeterminateAttemptId) + } } 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 16058de8bf3f..1061da7a3da2 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 @@ -47,6 +47,8 @@ private[spark] class SortShuffleWriter[K, V, C]( private val writeMetrics = context.taskMetrics().shuffleWriteMetrics + private val indeterminateAttemptId = context.getIndeterminateAttemptId + /** Write a bunch of records to this task's output */ override def write(records: Iterator[Product2[K, V]]): Unit = { sorter = if (dep.mapSideCombine) { @@ -64,12 +66,17 @@ 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 output = shuffleBlockResolver.getDataFile(dep.shuffleId, mapId) + val output = shuffleBlockResolver.getDataFile(dep.shuffleId, mapId, indeterminateAttemptId) val tmp = Utils.tempFileWith(output) try { - val blockId = ShuffleBlockId(dep.shuffleId, mapId, IndexShuffleBlockResolver.NOOP_REDUCE_ID) + val blockId = ShuffleBlockId( + dep.shuffleId, + mapId, + IndexShuffleBlockResolver.NOOP_REDUCE_ID, + indeterminateAttemptId) val partitionLengths = sorter.writePartitionedFile(blockId, tmp) - shuffleBlockResolver.writeIndexFileAndCommit(dep.shuffleId, mapId, partitionLengths, tmp) + shuffleBlockResolver.writeIndexFileAndCommit( + dep.shuffleId, mapId, partitionLengths, tmp, indeterminateAttemptId) mapStatus = MapStatus(blockManager.shuffleServerId, partitionLengths) } finally { if (tmp.exists() && !tmp.delete()) { 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 7ac2c71c18eb..8ce1e7e712f5 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockId.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockId.scala @@ -51,19 +51,49 @@ 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(). +// IndeterminateAttemptId is only set when the ShuffleMapStage's [[DeterministicLevel]] is +// INDETERMINATE and fetch fail triggered whole map stage rerun. @DeveloperApi -case class ShuffleBlockId(shuffleId: Int, mapId: Int, reduceId: Int) extends BlockId { - override def name: String = "shuffle_" + shuffleId + "_" + mapId + "_" + reduceId +case class ShuffleBlockId( + shuffleId: Int, + mapId: Int, + reduceId: Int, + indeterminateAttemptId: Option[Int] = None) + extends BlockId { + override def name: String = { + val nameStr = "shuffle_" + shuffleId + "_" + mapId + "_" + reduceId + if (indeterminateAttemptId.isEmpty) nameStr else nameStr + "_" + indeterminateAttemptId.get + } } @DeveloperApi -case class ShuffleDataBlockId(shuffleId: Int, mapId: Int, reduceId: Int) extends BlockId { - override def name: String = "shuffle_" + shuffleId + "_" + mapId + "_" + reduceId + ".data" +case class ShuffleDataBlockId( + shuffleId: Int, + mapId: Int, + reduceId: Int, + indeterminateAttemptId: Option[Int] = None) + extends BlockId { + override def name: String = { + val nameStr = "shuffle_" + shuffleId + "_" + mapId + "_" + reduceId + val nameStrWithIndeterminateAttempt = + if (indeterminateAttemptId.isEmpty) nameStr else nameStr + "_" + indeterminateAttemptId.get + nameStrWithIndeterminateAttempt + ".data" + } } @DeveloperApi -case class ShuffleIndexBlockId(shuffleId: Int, mapId: Int, reduceId: Int) extends BlockId { - override def name: String = "shuffle_" + shuffleId + "_" + mapId + "_" + reduceId + ".index" +case class ShuffleIndexBlockId( + shuffleId: Int, + mapId: Int, + reduceId: Int, + indeterminateAttemptId: Option[Int] = None) + extends BlockId { + override def name: String = { + val nameStr = "shuffle_" + shuffleId + "_" + mapId + "_" + reduceId + val nameStrWithIndeterminateAttempt = + if (indeterminateAttemptId.isEmpty) nameStr else nameStr + "_" + indeterminateAttemptId.get + nameStrWithIndeterminateAttempt + ".index" + } } @DeveloperApi @@ -106,6 +136,10 @@ object BlockId { val SHUFFLE = "shuffle_([0-9]+)_([0-9]+)_([0-9]+)".r val SHUFFLE_DATA = "shuffle_([0-9]+)_([0-9]+)_([0-9]+).data".r val SHUFFLE_INDEX = "shuffle_([0-9]+)_([0-9]+)_([0-9]+).index".r + // the extend shuffle/data/index is only used when INDETERMINATE stage rerun + val EXTEND_SHUFFLE = "shuffle_([0-9]+)_([0-9]+)_([0-9]+)_([0-9]+)".r + val EXTEND_SHUFFLE_DATA = "shuffle_([0-9]+)_([0-9]+)_([0-9]+)_([0-9]+).data".r + val EXTEND_SHUFFLE_INDEX = "shuffle_([0-9]+)_([0-9]+)_([0-9]+)_([0-9]+).index".r val BROADCAST = "broadcast_([0-9]+)([_A-Za-z0-9]*)".r val TASKRESULT = "taskresult_([0-9]+)".r val STREAM = "input-([0-9]+)-([0-9]+)".r @@ -122,6 +156,12 @@ object BlockId { ShuffleDataBlockId(shuffleId.toInt, mapId.toInt, reduceId.toInt) case SHUFFLE_INDEX(shuffleId, mapId, reduceId) => ShuffleIndexBlockId(shuffleId.toInt, mapId.toInt, reduceId.toInt) + case EXTEND_SHUFFLE(shuffleId, mapId, reduceId, stageAttemptId) => + ShuffleBlockId(shuffleId.toInt, mapId.toInt, reduceId.toInt, Some(stageAttemptId.toInt)) + case EXTEND_SHUFFLE_DATA(shuffleId, mapId, reduceId, stageAttemptId) => + ShuffleDataBlockId(shuffleId.toInt, mapId.toInt, reduceId.toInt, Some(stageAttemptId.toInt)) + case EXTEND_SHUFFLE_INDEX(shuffleId, mapId, reduceId, stageAttemptId) => + ShuffleIndexBlockId(shuffleId.toInt, mapId.toInt, reduceId.toInt, Some(stageAttemptId.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 c89d5cc971d2..69e7e4f89238 100644 --- a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala @@ -584,7 +584,7 @@ final class ShuffleBlockFetcherIterator( address: BlockManagerId, e: Throwable) = { blockId match { - case ShuffleBlockId(shufId, mapId, reduceId) => + case ShuffleBlockId(shufId, mapId, reduceId, _) => throw new FetchFailedException(address, shufId.toInt, mapId.toInt, reduceId, e) case _ => throw new SparkException( 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 88125a6b93ad..b6405b4bbbee 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 @@ -131,7 +131,8 @@ public void setUp() throws IOException { ); }); - when(shuffleBlockResolver.getDataFile(anyInt(), anyInt())).thenReturn(mergedOutputFile); + when(shuffleBlockResolver.getDataFile( + anyInt(), anyInt(), any())).thenReturn(mergedOutputFile); doAnswer(invocationOnMock -> { partitionSizesInMergedFile = (long[]) invocationOnMock.getArguments()[2]; File tmp = (File) invocationOnMock.getArguments()[3]; @@ -139,7 +140,8 @@ public void setUp() throws IOException { tmp.renameTo(mergedOutputFile); return null; }).when(shuffleBlockResolver) - .writeIndexFileAndCommit(anyInt(), anyInt(), any(long[].class), any(File.class)); + .writeIndexFileAndCommit( + anyInt(), anyInt(), any(long[].class), any(File.class), any(Option.class)); when(diskBlockManager.createTempShuffleBlock()).thenAnswer(invocationOnMock -> { TempShuffleBlockId blockId = new TempShuffleBlockId(UUID.randomUUID()); diff --git a/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala b/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala index c16e227edbfa..a66f433bd118 100644 --- a/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala @@ -533,8 +533,8 @@ class CleanerTester( private def getShuffleBlocks(shuffleId: Int): Seq[BlockId] = { blockManager.master.getMatchingBlockIds( _ match { - case ShuffleBlockId(`shuffleId`, _, _) => true - case ShuffleIndexBlockId(`shuffleId`, _, _) => true + case ShuffleBlockId(`shuffleId`, _, _, _) => true + case ShuffleIndexBlockId(`shuffleId`, _, _, _) => true case _ => false }, askSlaves = true) } 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 6d2ef17a7a79..0f0312d7be7f 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderSuite.scala @@ -25,6 +25,7 @@ import org.mockito.Mockito.{mock, when} import org.apache.spark._ import org.apache.spark.internal.config import org.apache.spark.network.buffer.{ManagedBuffer, NioManagedBuffer} +import org.apache.spark.rdd.{DeterministicLevel, RDD} import org.apache.spark.serializer.{JavaSerializer, SerializerManager} import org.apache.spark.storage.{BlockManager, BlockManagerId, ShuffleBlockId} @@ -115,9 +116,12 @@ class BlockStoreShuffleReaderSuite extends SparkFunSuite with LocalSparkContext // Create a mocked shuffle handle to pass into HashShuffleReader. val shuffleHandle = { val dependency = mock(classOf[ShuffleDependency[Int, Int, Int]]) + val rdd = mock(classOf[RDD[Product2[Int, Int]]]) + when(rdd.outputDeterministicLevel).thenReturn(DeterministicLevel.DETERMINATE) when(dependency.serializer).thenReturn(serializer) when(dependency.aggregator).thenReturn(None) when(dependency.keyOrdering).thenReturn(None) + when(dependency.rdd).thenReturn(rdd) new BaseShuffleHandle(shuffleId, numMaps, dependency) } 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 fc1422dfaac7..57a71b007264 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 @@ -67,7 +67,8 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte when(dependency.partitioner).thenReturn(new HashPartitioner(7)) when(dependency.serializer).thenReturn(new JavaSerializer(conf)) when(taskContext.taskMetrics()).thenReturn(taskMetrics) - when(blockResolver.getDataFile(0, 0)).thenReturn(outputFile) + when(taskContext.getIndeterminateAttemptId).thenReturn(None) + when(blockResolver.getDataFile(0, 0, None)).thenReturn(outputFile) doAnswer { (invocationOnMock: InvocationOnMock) => val tmp = invocationOnMock.getArguments()(3).asInstanceOf[File] if (tmp != null) { @@ -76,7 +77,8 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte } null }.when(blockResolver) - .writeIndexFileAndCommit(anyInt, anyInt, any(classOf[Array[Long]]), any(classOf[File])) + .writeIndexFileAndCommit( + anyInt, anyInt, any(classOf[Array[Long]]), any(classOf[File]), any(classOf[Option[Int]])) when(blockManager.diskBlockManager).thenReturn(diskBlockManager) when(blockManager.getDiskWriter( any[BlockId], 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 ff4755833a91..7bf510795a05 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockIdSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockIdSuite.scala @@ -90,6 +90,48 @@ class BlockIdSuite extends SparkFunSuite { assertSame(id, BlockId(id.toString)) } + test("extend shuffle") { + val id = ShuffleBlockId(1, 2, 3, Some(4)) + assertSame(id, ShuffleBlockId(1, 2, 3, Some(4))) + assertDifferent(id, ShuffleBlockId(3, 2, 3, Some(5))) + assert(id.name === "shuffle_1_2_3_4") + assert(id.asRDDId === None) + assert(id.shuffleId === 1) + assert(id.mapId === 2) + assert(id.reduceId === 3) + assert(id.indeterminateAttemptId.get === 4) + assert(id.isShuffle) + assertSame(id, BlockId(id.toString)) + } + + test("extend shuffle data") { + val id = ShuffleDataBlockId(4, 5, 6, Some(7)) + assertSame(id, ShuffleDataBlockId(4, 5, 6, Some(7))) + assertDifferent(id, ShuffleDataBlockId(6, 5, 6, Some(8))) + assert(id.name === "shuffle_4_5_6_7.data") + assert(id.asRDDId === None) + assert(id.shuffleId === 4) + assert(id.mapId === 5) + assert(id.reduceId === 6) + assert(id.indeterminateAttemptId.get === 7) + assert(!id.isShuffle) + assertSame(id, BlockId(id.toString)) + } + + test("extend shuffle index") { + val id = ShuffleIndexBlockId(7, 8, 9, Some(10)) + assertSame(id, ShuffleIndexBlockId(7, 8, 9, Some(10))) + assertDifferent(id, ShuffleIndexBlockId(9, 8, 9, Some(11))) + assert(id.name === "shuffle_7_8_9_10.index") + assert(id.asRDDId === None) + assert(id.shuffleId === 7) + assert(id.mapId === 8) + assert(id.reduceId === 9) + assert(id.indeterminateAttemptId.get === 10) + assert(!id.isShuffle) + assertSame(id, BlockId(id.toString)) + } + test("broadcast") { val id = BroadcastBlockId(42) assertSame(id, BroadcastBlockId(42)) From 1523d9d4ff272a7fe311a47ae10f2f78bafa7a64 Mon Sep 17 00:00:00 2001 From: liyuanjian Date: Mon, 19 Nov 2018 11:58:17 +0800 Subject: [PATCH 02/16] Add mapSideIndeterminate in shuffle dependency --- core/src/main/scala/org/apache/spark/Dependency.scala | 5 ++++- .../org/apache/spark/shuffle/BlockStoreShuffleReader.scala | 5 +---- .../apache/spark/shuffle/BlockStoreShuffleReaderSuite.scala | 4 ---- 3 files changed, 5 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Dependency.scala b/core/src/main/scala/org/apache/spark/Dependency.scala index fb051a8c0db8..028047704cf8 100644 --- a/core/src/main/scala/org/apache/spark/Dependency.scala +++ b/core/src/main/scala/org/apache/spark/Dependency.scala @@ -20,7 +20,7 @@ package org.apache.spark import scala.reflect.ClassTag import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.rdd.RDD +import org.apache.spark.rdd.{DeterministicLevel, RDD} import org.apache.spark.serializer.Serializer import org.apache.spark.shuffle.{ShuffleHandle, ShuffleWriteProcessor} @@ -95,6 +95,9 @@ class ShuffleDependency[K: ClassTag, V: ClassTag, C: ClassTag]( val shuffleHandle: ShuffleHandle = _rdd.context.env.shuffleManager.registerShuffle( shuffleId, _rdd.partitions.length, this) + val mapSideIndeterminate: Boolean = + _rdd.outputDeterministicLevel == DeterministicLevel.INDETERMINATE + _rdd.sparkContext.cleaner.foreach(_.registerShuffleForCleanup(this)) } 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 d3029cd86ced..e1311401844b 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala @@ -42,9 +42,6 @@ private[spark] class BlockStoreShuffleReader[K, C]( private val dep = handle.dependency - private val mapSideIndeterminate = - dep.rdd.outputDeterministicLevel == DeterministicLevel.INDETERMINATE - /** Read the combined key-values for this reduce task */ override def read(): Iterator[Product2[K, C]] = { val wrappedStreams = new ShuffleBlockFetcherIterator( @@ -52,7 +49,7 @@ private[spark] class BlockStoreShuffleReader[K, C]( blockManager.shuffleClient, blockManager, mapOutputTracker.getMapSizesByExecutorId( - handle.shuffleId, startPartition, endPartition, mapSideIndeterminate), + handle.shuffleId, startPartition, endPartition, dep.mapSideIndeterminate), 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/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderSuite.scala index 0f0312d7be7f..6d2ef17a7a79 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderSuite.scala @@ -25,7 +25,6 @@ import org.mockito.Mockito.{mock, when} import org.apache.spark._ import org.apache.spark.internal.config import org.apache.spark.network.buffer.{ManagedBuffer, NioManagedBuffer} -import org.apache.spark.rdd.{DeterministicLevel, RDD} import org.apache.spark.serializer.{JavaSerializer, SerializerManager} import org.apache.spark.storage.{BlockManager, BlockManagerId, ShuffleBlockId} @@ -116,12 +115,9 @@ class BlockStoreShuffleReaderSuite extends SparkFunSuite with LocalSparkContext // Create a mocked shuffle handle to pass into HashShuffleReader. val shuffleHandle = { val dependency = mock(classOf[ShuffleDependency[Int, Int, Int]]) - val rdd = mock(classOf[RDD[Product2[Int, Int]]]) - when(rdd.outputDeterministicLevel).thenReturn(DeterministicLevel.DETERMINATE) when(dependency.serializer).thenReturn(serializer) when(dependency.aggregator).thenReturn(None) when(dependency.keyOrdering).thenReturn(None) - when(dependency.rdd).thenReturn(rdd) new BaseShuffleHandle(shuffleId, numMaps, dependency) } From a0532523cc0037a4f301f3bea5f93c8a32897f3a Mon Sep 17 00:00:00 2001 From: liyuanjian Date: Mon, 19 Nov 2018 23:56:28 +0800 Subject: [PATCH 03/16] Rerun all indeterminate stage --- .../org/apache/spark/MapOutputTracker.scala | 14 ++++++++++++++ .../apache/spark/scheduler/DAGScheduler.scala | 17 +++++++++-------- .../apache/spark/scheduler/ResultStage.scala | 4 ++++ .../spark/scheduler/ShuffleMapStage.scala | 9 ++++++++- .../org/apache/spark/scheduler/Stage.scala | 17 +++++++---------- .../shuffle/IndexShuffleBlockResolver.scala | 8 -------- 6 files changed, 42 insertions(+), 27 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index a6c3c92d0cdd..468708913de7 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -236,6 +236,13 @@ private class ShuffleStatus(numPartitions: Int) { def indeterminateAttemptId: Option[Int] = synchronized { _indeterminateAttemptId } + + /** + * Reset the indeterminate attempt id to None. + */ + def resetIndeterminateAttemptId(): Unit = synchronized { + _indeterminateAttemptId = None + } } private[spark] sealed trait MapOutputTrackerMessage @@ -712,6 +719,10 @@ private[spark] class MapOutputTrackerMaster( shuffleStatuses(shuffleId).setAndCheckIndeterminateAttemptId(stageAttemptId) } + def unregisterIndeterminateShuffle(shuffleId: Int): Unit = { + shuffleStatuses(shuffleId).resetIndeterminateAttemptId() + } + override def stop() { mapOutputRequests.offer(PoisonPill) threadpool.shutdown() @@ -768,6 +779,8 @@ private[spark] class MapOutputTrackerWorker(conf: SparkConf) extends MapOutputTr if (id == null) { val fetchedId = askTracker[Option[Int]](GetIndeterminateAttemptId(shuffleId)) indeterminateAttemptIds.put(shuffleId, fetchedId) + logInfo(s"Getting the indeterminate attempt id $id for" + + s" shuffleId: $shuffleId, and update local cache.") fetchedId } else { id @@ -853,6 +866,7 @@ private[spark] class MapOutputTrackerWorker(conf: SparkConf) extends MapOutputTr logInfo("Updating epoch to " + newEpoch + " and clearing cache") epoch = newEpoch mapStatuses.clear() + indeterminateAttemptIds.clear() } } } 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 69ba5886a778..ca91ef6459e4 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1206,7 +1206,7 @@ private[spark] class DAGScheduler( new ShuffleMapTask(stage.id, stage.latestInfo.attemptNumber, taskBinary, part, locs, properties, serializedTaskMetrics, Option(jobId), Option(sc.applicationId), sc.applicationAttemptId, stage.rdd.isBarrier(), - stage.isDeterminate()) + !stage.isIndeterminate()) } case stage: ResultStage => @@ -1451,6 +1451,10 @@ private[spark] class DAGScheduler( // available. mapOutputTracker.registerMapOutput( shuffleStage.shuffleDep.shuffleId, smt.partitionId, status) + if (stage.isIndeterminate()) { + mapOutputTracker.registerIndeterminateShuffle( + shuffleStage.shuffleDep.shuffleId, smt.stageAttemptId) + } } if (runningStages.contains(shuffleStage) && shuffleStage.pendingPartitions.isEmpty) { @@ -1559,7 +1563,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 `failedStage`, 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 @@ -1591,12 +1595,9 @@ private[spark] class DAGScheduler( case mapStage: ShuffleMapStage => val numMissingPartitions = mapStage.findMissingPartitions().length if (numMissingPartitions < mapStage.numTasks) { - // Mark all the map as broken in the map stage, to ensure retry all the - // tasks on resubmitted stage attempt. Also mark this map stage as - // indeterminate, will rerun all its task and generate new intermediate - // shuffle file. - mapOutputTracker.unregisterAllMapOutput(mapStage.shuffleDep.shuffleId) - mapStage.markAsInDeterminate() + 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/ResultStage.scala b/core/src/main/scala/org/apache/spark/scheduler/ResultStage.scala index d1687830ff7b..3728cc13eb6f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ResultStage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ResultStage.scala @@ -55,11 +55,15 @@ private[spark] class ResultStage( /** * Returns the sequence of partition ids that are missing (i.e. needs to be computed). + * If the current stage is indeterminate, missing partition is all partitions every time. * * This can only be called when there is an active job. */ override def findMissingPartitions(): Seq[Int] = { val job = activeJob.get + if (isIndeterminate()) { + job.resetAllPartitions() + } (0 until job.numPartitions).filter(id => !job.finished(id)) } diff --git a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapStage.scala b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapStage.scala index 1b44d0aee319..67ec6ee2a2b3 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapStage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapStage.scala @@ -87,8 +87,15 @@ private[spark] class ShuffleMapStage( */ def isAvailable: Boolean = numAvailableOutputs == numPartitions - /** Returns the sequence of partition ids that are missing (i.e. needs to be computed). */ + /** + * Returns the sequence of partition ids that are missing (i.e. needs to be computed). + * If the current stage is indeterminate, missing partition is all partitions every time. + */ override def findMissingPartitions(): Seq[Int] = { + if (isIndeterminate()) { + mapOutputTrackerMaster.unregisterIndeterminateShuffle(shuffleDep.shuffleId) + mapOutputTrackerMaster.unregisterAllMapOutput(shuffleDep.shuffleId) + } mapOutputTrackerMaster .findMissingPartitions(shuffleDep.shuffleId) .getOrElse(0 until numPartitions) 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 1017a91d82c8..e12718dc8279 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 /** @@ -89,9 +89,6 @@ private[scheduler] abstract class Stage( */ val failedAttemptIds = new HashSet[Int] - /** The identification of this stage is a determinate stage and being retried. */ - private var determinate: Boolean = true - private[scheduler] def clearFailures() : Unit = { failedAttemptIds.clear() } @@ -117,12 +114,12 @@ private[scheduler] abstract class Stage( case _ => false } - /** Returns the sequence of partition ids that are missing (i.e. needs to be computed). */ + /** + * Returns the sequence of partition ids that are missing (i.e. needs to be computed). + * If the current stage is indeterminate, missing partition is all partitions every time. + */ def findMissingPartitions(): Seq[Int] - def markAsInDeterminate(): Unit = { - determinate = false - } - - def isDeterminate(): Boolean = determinate + def isIndeterminate(): Boolean = + rdd.outputDeterministicLevel == DeterministicLevel.INDETERMINATE } 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 49dd69a0d908..9ec8fd9d1c8a 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala @@ -51,17 +51,10 @@ private[spark] class IndexShuffleBlockResolver( private val transportConf = SparkTransportConf.fromSparkConf(conf, "shuffle") - private def checkStageAttemptId(stageAttemptId: Option[Int]): Unit = { - require(stageAttemptId.isEmpty || (stageAttemptId.isDefined && stageAttemptId.get > 0), - "Setting stage attempt id here represent for rerunning indeterminate stage/task," + - "so it must > 0, otherwise it kept None as default.") - } - def getDataFile( shuffleId: Int, mapId: Int, stageAttemptId: Option[Int] = None): File = { - checkStageAttemptId(stageAttemptId) blockManager.diskBlockManager.getFile( ShuffleDataBlockId(shuffleId, mapId, NOOP_REDUCE_ID, stageAttemptId)) } @@ -70,7 +63,6 @@ private[spark] class IndexShuffleBlockResolver( shuffleId: Int, mapId: Int, stageAttemptId: Option[Int] = None): File = { - checkStageAttemptId(stageAttemptId) blockManager.diskBlockManager.getFile( ShuffleIndexBlockId(shuffleId, mapId, NOOP_REDUCE_ID, stageAttemptId)) } From bf01951fb8e43378958715d1a6f85d37c55b904c Mon Sep 17 00:00:00 2001 From: liyuanjian Date: Tue, 20 Nov 2018 15:46:58 +0800 Subject: [PATCH 04/16] UT fix --- .../org/apache/spark/MapOutputTracker.scala | 4 +-- .../scala/org/apache/spark/TaskContext.scala | 6 ++-- .../main/scala/org/apache/spark/rdd/RDD.scala | 2 +- .../apache/spark/scheduler/DAGScheduler.scala | 11 ++++--- .../apache/spark/scheduler/ResultStage.scala | 4 --- .../spark/scheduler/ShuffleMapStage.scala | 15 +++++++--- .../spark/scheduler/ShuffleMapTask.scala | 4 +-- .../org/apache/spark/scheduler/Stage.scala | 13 +++++---- .../org/apache/spark/scheduler/Task.scala | 6 ++-- .../spark/scheduler/DAGSchedulerSuite.scala | 29 +++++++++++++++++-- pom.xml | 1 + 11 files changed, 64 insertions(+), 31 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 468708913de7..a4072516f8ba 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -238,8 +238,8 @@ private class ShuffleStatus(numPartitions: Int) { } /** - * Reset the indeterminate attempt id to None. - */ + * Reset the indeterminate attempt id to None. + */ def resetIndeterminateAttemptId(): Unit = synchronized { _indeterminateAttemptId = None } diff --git a/core/src/main/scala/org/apache/spark/TaskContext.scala b/core/src/main/scala/org/apache/spark/TaskContext.scala index 919fac2836a0..1ec471b1ac36 100644 --- a/core/src/main/scala/org/apache/spark/TaskContext.scala +++ b/core/src/main/scala/org/apache/spark/TaskContext.scala @@ -233,10 +233,8 @@ abstract class TaskContext extends Serializable { private[spark] def getLocalProperties: Properties /** - * Get the indeterminate stage attempt id, it will return None in following two case: - * 1. The task is determinate. - * 2. The task is indeterminate but no retry occurred. - * Otherwise it return the stage attempt number. + * Get the indeterminate stage attempt id, it will return None if the task is determinate. + * While the task is indeterminate it return the stage attempt number. */ private[spark] def getIndeterminateAttemptId: Option[Int] } diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index 1b67e9906457..2ca9fa871206 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -1736,7 +1736,7 @@ abstract class RDD[T: ClassTag]( } // Avoid handling doCheckpoint multiple times to prevent excessive recursion - @transient private var doCheckpointCalled = false + @transient private[spark] var doCheckpointCalled = false /** * Performs the checkpointing of this RDD by saving this. It is called after a job using this RDD 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 ca91ef6459e4..25a331c094a7 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1098,7 +1098,10 @@ 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. + stage.clearIntermediateState() + + // 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 @@ -1206,7 +1209,7 @@ private[spark] class DAGScheduler( new ShuffleMapTask(stage.id, stage.latestInfo.attemptNumber, taskBinary, part, locs, properties, serializedTaskMetrics, Option(jobId), Option(sc.applicationId), sc.applicationAttemptId, stage.rdd.isBarrier(), - !stage.isIndeterminate()) + stage.isIndeterminate) } case stage: ResultStage => @@ -1451,7 +1454,7 @@ private[spark] class DAGScheduler( // available. mapOutputTracker.registerMapOutput( shuffleStage.shuffleDep.shuffleId, smt.partitionId, status) - if (stage.isIndeterminate()) { + if (stage.isIndeterminate) { mapOutputTracker.registerIndeterminateShuffle( shuffleStage.shuffleDep.shuffleId, smt.stageAttemptId) } @@ -1563,7 +1566,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 `failedStage`, 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/ResultStage.scala b/core/src/main/scala/org/apache/spark/scheduler/ResultStage.scala index 3728cc13eb6f..d1687830ff7b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ResultStage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ResultStage.scala @@ -55,15 +55,11 @@ private[spark] class ResultStage( /** * Returns the sequence of partition ids that are missing (i.e. needs to be computed). - * If the current stage is indeterminate, missing partition is all partitions every time. * * This can only be called when there is an active job. */ override def findMissingPartitions(): Seq[Int] = { val job = activeJob.get - if (isIndeterminate()) { - job.resetAllPartitions() - } (0 until job.numPartitions).filter(id => !job.finished(id)) } diff --git a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapStage.scala b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapStage.scala index 67ec6ee2a2b3..62660fd46026 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapStage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapStage.scala @@ -92,12 +92,19 @@ private[spark] class ShuffleMapStage( * If the current stage is indeterminate, missing partition is all partitions every time. */ override def findMissingPartitions(): Seq[Int] = { - if (isIndeterminate()) { - mapOutputTrackerMaster.unregisterIndeterminateShuffle(shuffleDep.shuffleId) - mapOutputTrackerMaster.unregisterAllMapOutput(shuffleDep.shuffleId) - } mapOutputTrackerMaster .findMissingPartitions(shuffleDep.shuffleId) .getOrElse(0 until numPartitions) } + + /** + * Clear the intermediate status including shuffle status and indeterminate attempt id for + * shuffle map stage. + */ + override def clearIntermediateState(): Unit = { + if (isIndeterminate) { + mapOutputTrackerMaster.unregisterIndeterminateShuffle(shuffleDep.shuffleId) + mapOutputTrackerMaster.unregisterAllMapOutput(shuffleDep.shuffleId) + } + } } 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 59159b54beee..43a693961fcb 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala @@ -64,9 +64,9 @@ private[spark] class ShuffleMapTask( appId: Option[String] = None, appAttemptId: Option[String] = None, isBarrier: Boolean = false, - isDeterminate: Boolean = true) + isIndeterminate: Boolean = true) extends Task[MapStatus](stageId, stageAttemptId, partition.index, localProperties, - serializedTaskMetrics, jobId, appId, appAttemptId, isBarrier, isDeterminate) + serializedTaskMetrics, jobId, appId, appAttemptId, isBarrier, isIndeterminate) with Logging { /** A constructor used only in test suites. This does not require passing in an RDD. */ 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 e12718dc8279..89018affdb12 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala @@ -114,12 +114,15 @@ private[scheduler] abstract class Stage( case _ => false } - /** - * Returns the sequence of partition ids that are missing (i.e. needs to be computed). - * If the current stage is indeterminate, missing partition is all partitions every time. - */ + /** 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 + + /** + * Clear the intermediate status (e.g. map status, success task, etc) for the stage, currently + * we call the function before indeterminate stage rerunning. + */ + def clearIntermediateState(): Unit = {} } diff --git a/core/src/main/scala/org/apache/spark/scheduler/Task.scala b/core/src/main/scala/org/apache/spark/scheduler/Task.scala index 9e9b3cb7d37a..b11047e4c788 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Task.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Task.scala @@ -51,7 +51,7 @@ import org.apache.spark.util._ * @param appAttemptId attempt id of the app this task belongs to * @param isBarrier whether this task belongs to a barrier stage. Spark must launch all the tasks * at the same time for a barrier stage. - * @param isDeterminate whether this task is determinate. + * @param isIndeterminate whether this task is determinate. */ private[spark] abstract class Task[T]( val stageId: Int, @@ -65,7 +65,7 @@ private[spark] abstract class Task[T]( val appId: Option[String] = None, val appAttemptId: Option[String] = None, val isBarrier: Boolean = false, - val isDeterminate: Boolean = true) extends Serializable { + val isIndeterminate: Boolean = false) extends Serializable { @transient lazy val metrics: TaskMetrics = SparkEnv.get.closureSerializer.newInstance().deserialize(ByteBuffer.wrap(serializedTaskMetrics)) @@ -101,7 +101,7 @@ private[spark] abstract class Task[T]( taskContext } - if (!isDeterminate) context.markAsIndeterminate() + if (isIndeterminate) context.markAsIndeterminate() TaskContext.setTaskContext(context) taskThread = Thread.currentThread() 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 72c20a817336..21e85f441a5a 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -2750,8 +2750,33 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi FetchFailed(makeBlockManagerId("hostA"), shuffleId1, 0, 0, "ignored"), null)) - // The job should fail because Spark can't rollback the shuffle map stage. - assert(failure != null && failure.getMessage.contains("Spark cannot rollback")) + 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() } private def assertResultStageFailToRollback(mapRdd: MyRDD): Unit = { diff --git a/pom.xml b/pom.xml index 5879a76477dd..31d2948ec601 100644 --- a/pom.xml +++ b/pom.xml @@ -2149,6 +2149,7 @@ net.alchim31.maven scala-maven-plugin + 3.4.4 From 3dbb614c614dc2461571333182b74a9a74d080c8 Mon Sep 17 00:00:00 2001 From: liyuanjian Date: Tue, 20 Nov 2018 19:28:07 +0800 Subject: [PATCH 05/16] Fix mima test --- .../main/scala/org/apache/spark/rdd/RDD.scala | 2 +- project/MimaExcludes.scala | 16 ++++++++++++++++ 2 files changed, 17 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index 2ca9fa871206..1b67e9906457 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -1736,7 +1736,7 @@ abstract class RDD[T: ClassTag]( } // Avoid handling doCheckpoint multiple times to prevent excessive recursion - @transient private[spark] var doCheckpointCalled = false + @transient private var doCheckpointCalled = false /** * Performs the checkpointing of this RDD by saving this. It is called after a job using this RDD diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 2a5d01e3772c..9f3aaf2f34b0 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -270,6 +270,22 @@ object MimaExcludes { // [SPARK-26457] Show hadoop configurations in HistoryServer environment tab ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.status.api.v1.ApplicationEnvironmentInfo.this"), + // [SPARK-25341][CORE] Support rolling back a shuffle map stage and re-generate the shuffle files + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.TaskContext.getIndeterminateAttemptId"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.TaskContext.markAsIndeterminate"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.storage.ShuffleIndexBlockId.copy"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.storage.ShuffleIndexBlockId.this"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.storage.ShuffleDataBlockId.copy"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.storage.ShuffleDataBlockId.this"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.storage.ShuffleBlockId.copy"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.storage.ShuffleBlockId.this"), + ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.storage.ShuffleIndexBlockId$"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.storage.ShuffleIndexBlockId.apply"), + ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.storage.ShuffleDataBlockId$"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.storage.ShuffleDataBlockId.apply"), + ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.storage.ShuffleBlockId$"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.storage.ShuffleBlockId.apply"), + // Data Source V2 API changes (problem: Problem) => problem match { case MissingClassProblem(cls) => From b9874e06f40fec5351037e2938e65931aaeb994b Mon Sep 17 00:00:00 2001 From: liyuanjian Date: Fri, 23 Nov 2018 10:49:59 +0800 Subject: [PATCH 06/16] Add new UT --- .../shuffle/ExternalShuffleBlockHandler.java | 11 +--- .../ExternalShuffleBlockResolverSuite.java | 26 +++++++- .../shuffle/ExternalShuffleCleanupSuite.java | 2 +- .../ExternalShuffleIntegrationSuite.java | 25 +++++++- .../shuffle/NonShuffleFilesCleanupSuite.java | 2 +- .../shuffle/TestShuffleDataContext.java | 4 +- .../spark/ExternalShuffleServiceSuite.scala | 17 ++++- .../apache/spark/MapOutputTrackerSuite.scala | 49 ++++++++++++++ .../scala/org/apache/spark/ShuffleSuite.scala | 22 +++++++ .../spark/scheduler/DAGSchedulerSuite.scala | 64 ++++++++++++++++++- .../sort/IndexShuffleBlockResolverSuite.scala | 29 ++++++--- 11 files changed, 227 insertions(+), 24 deletions(-) diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java index 95737e8dcb62..648aa94e928a 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java @@ -218,7 +218,7 @@ private class ManagedBufferIterator implements Iterator { private final int shuffleId; // An array containing mapId and reduceId pairs. private final int[] mapIdAndReduceIds; - private final int indeterminateRetryId; + private final int indeterminateAttemptId; ManagedBufferIterator(String appId, String execId, String[] blockIds) { this.appId = appId; @@ -227,7 +227,7 @@ private class ManagedBufferIterator implements Iterator { checkBlockId(blockId0Parts, blockIds, 0); this.shuffleId = Integer.parseInt(blockId0Parts[1]); mapIdAndReduceIds = new int[2 * blockIds.length]; - this.indeterminateRetryId = + this.indeterminateAttemptId = (blockId0Parts.length == 5) ? Integer.parseInt(blockId0Parts[4]) : -1; for (int i = 0; i < blockIds.length; i++) { String[] blockIdParts = blockIds[i].split("_"); @@ -247,11 +247,6 @@ private void checkBlockId(String[] blockIdParts, String[] wholeBlockId, int inde throw new IllegalArgumentException( "Unexpected shuffle block id format: " + wholeBlockId[index]); } - if (blockIdParts.length == 5 && blockIdParts[4].equals("0")) { - throw new IllegalArgumentException( - "Unexpected indeterminateRetryId in shuffle block id, if indeterminateRetryId be set," + - " it should only up to 0: " + wholeBlockId[index]); - } } @Override @@ -262,7 +257,7 @@ public boolean hasNext() { @Override public ManagedBuffer next() { final ManagedBuffer block = blockManager.getBlockData(appId, execId, shuffleId, - mapIdAndReduceIds[index], mapIdAndReduceIds[index + 1], indeterminateRetryId); + mapIdAndReduceIds[index], mapIdAndReduceIds[index + 1], indeterminateAttemptId); index += 2; metrics.blockTransferRateBytes.mark(block != null ? block.size() : 0); return block; diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolverSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolverSuite.java index 459629c5f05f..3b7cee2b0145 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolverSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolverSuite.java @@ -53,7 +53,10 @@ public static void beforeAll() throws IOException { // Write some sort data. dataContext.insertSortShuffleData(0, 0, new byte[][] { sortBlock0.getBytes(StandardCharsets.UTF_8), - sortBlock1.getBytes(StandardCharsets.UTF_8)}); + sortBlock1.getBytes(StandardCharsets.UTF_8)}, false); + dataContext.insertSortShuffleData(0, 0, new byte[][] { + sortBlock0.getBytes(StandardCharsets.UTF_8), + sortBlock1.getBytes(StandardCharsets.UTF_8)}, true); } @AfterClass @@ -113,6 +116,27 @@ public void testSortShuffleBlocks() throws IOException { } } + @Test + public void testExtendedSortShuffleBlocks() throws IOException { + ExternalShuffleBlockResolver resolver = new ExternalShuffleBlockResolver(conf, null); + resolver.registerExecutor("app0", "exec0", + dataContext.createExecutorInfo(SORT_MANAGER)); + + try (InputStream block0Stream = resolver.getBlockData( + "app0", "exec0", 0, 0, 0, 0).createInputStream()) { + String block0 = + CharStreams.toString(new InputStreamReader(block0Stream, StandardCharsets.UTF_8)); + assertEquals(sortBlock0, block0); + } + + try (InputStream block1Stream = resolver.getBlockData( + "app0", "exec0", 0, 0, 1, 0).createInputStream()) { + String block1 = + CharStreams.toString(new InputStreamReader(block1Stream, StandardCharsets.UTF_8)); + assertEquals(sortBlock1, block1); + } + } + @Test public void jsonSerializationOfExecutorRegistration() throws IOException { ObjectMapper mapper = new ObjectMapper(); diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleCleanupSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleCleanupSuite.java index 47c087088a8a..259cc2e04b7d 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleCleanupSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleCleanupSuite.java @@ -142,7 +142,7 @@ private static TestShuffleDataContext createSomeData() throws IOException { dataContext.create(); dataContext.insertSortShuffleData(rand.nextInt(1000), rand.nextInt(1000), new byte[][] { "ABC".getBytes(StandardCharsets.UTF_8), - "DEF".getBytes(StandardCharsets.UTF_8)}); + "DEF".getBytes(StandardCharsets.UTF_8)}, false); return dataContext; } } diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java index f5b1ec9d46da..98418cf86ea7 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java @@ -84,7 +84,8 @@ public static void beforeAll() throws IOException { dataContext0 = new TestShuffleDataContext(2, 5); dataContext0.create(); - dataContext0.insertSortShuffleData(0, 0, exec0Blocks); + dataContext0.insertSortShuffleData(0, 0, exec0Blocks, false); + dataContext0.insertSortShuffleData(0, 0, exec0Blocks, true); conf = new TransportConf("shuffle", MapConfigProvider.EMPTY); handler = new ExternalShuffleBlockHandler(conf, null); @@ -191,6 +192,28 @@ public void testFetchThreeSort() throws Exception { exec0Fetch.releaseBuffers(); } + @Test + public void testFetchOneExtendedSort() throws Exception { + registerExecutor("exec-0", dataContext0.createExecutorInfo(SORT_MANAGER)); + FetchResult exec0Fetch = fetchBlocks("exec-0", new String[] { "shuffle_0_0_0_0" }); + assertEquals(Sets.newHashSet("shuffle_0_0_0_0"), exec0Fetch.successBlocks); + assertTrue(exec0Fetch.failedBlocks.isEmpty()); + assertBufferListsEqual(exec0Fetch.buffers, Arrays.asList(exec0Blocks[0])); + exec0Fetch.releaseBuffers(); + } + + @Test + public void testFetchThreeExtendedSort() throws Exception { + registerExecutor("exec-0", dataContext0.createExecutorInfo(SORT_MANAGER)); + FetchResult exec0Fetch = fetchBlocks("exec-0", + new String[] { "shuffle_0_0_0_0", "shuffle_0_0_1_0", "shuffle_0_0_2_0" }); + assertEquals(Sets.newHashSet("shuffle_0_0_0_0", "shuffle_0_0_1_0", "shuffle_0_0_2_0"), + exec0Fetch.successBlocks); + assertTrue(exec0Fetch.failedBlocks.isEmpty()); + assertBufferListsEqual(exec0Fetch.buffers, Arrays.asList(exec0Blocks)); + exec0Fetch.releaseBuffers(); + } + @Test (expected = RuntimeException.class) public void testRegisterInvalidExecutor() throws Exception { registerExecutor("exec-1", dataContext0.createExecutorInfo("unknown sort manager")); diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/NonShuffleFilesCleanupSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/NonShuffleFilesCleanupSuite.java index d22f3ace4103..80b27010d903 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/NonShuffleFilesCleanupSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/NonShuffleFilesCleanupSuite.java @@ -211,7 +211,7 @@ private static void createShuffleFiles(TestShuffleDataContext dataContext) throw Random rand = new Random(123); dataContext.insertSortShuffleData(rand.nextInt(1000), rand.nextInt(1000), new byte[][] { "ABC".getBytes(StandardCharsets.UTF_8), - "DEF".getBytes(StandardCharsets.UTF_8)}); + "DEF".getBytes(StandardCharsets.UTF_8)}, false); } private static void createNonShuffleFiles(TestShuffleDataContext dataContext) throws IOException { diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/TestShuffleDataContext.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/TestShuffleDataContext.java index 6989c3baf2e2..f0904fbc42a1 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/TestShuffleDataContext.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/TestShuffleDataContext.java @@ -68,8 +68,10 @@ public void cleanup() { } /** Creates reducer blocks in a sort-based data format within our local dirs. */ - public void insertSortShuffleData(int shuffleId, int mapId, byte[][] blocks) throws IOException { + public void insertSortShuffleData( + int shuffleId, int mapId, byte[][] blocks, boolean extendedBlockId) throws IOException { String blockId = "shuffle_" + shuffleId + "_" + mapId + "_0"; + if (extendedBlockId) blockId += "_0"; OutputStream dataStream = null; DataOutputStream indexStream = null; diff --git a/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala b/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala index 8b737cd8c81f..6c0d7cefc37f 100644 --- a/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala +++ b/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala @@ -25,6 +25,7 @@ import org.apache.spark.network.netty.SparkTransportConf import org.apache.spark.network.server.TransportServer import org.apache.spark.network.shuffle.{ExternalShuffleBlockHandler, ExternalShuffleClient} import org.apache.spark.util.Utils +import org.apache.spark.rdd.RDD /** * This suite creates an external shuffle server and routes all shuffle fetches through it. @@ -63,7 +64,7 @@ class ExternalShuffleServiceSuite extends ShuffleSuite with BeforeAndAfterAll { } // This test ensures that the external shuffle service is actually in use for the other tests. - test("using external shuffle service") { + private def checkResultWithShuffleService(createRDD: (SparkContext => RDD[_])): Unit = { sc = new SparkContext("local-cluster[2,1,1024]", "test", conf) sc.env.blockManager.externalShuffleServiceEnabled should equal(true) sc.env.blockManager.shuffleClient.getClass should equal(classOf[ExternalShuffleClient]) @@ -76,7 +77,7 @@ class ExternalShuffleServiceSuite extends ShuffleSuite with BeforeAndAfterAll { // Therefore, we should wait until all slaves are up TestUtils.waitUntilExecutorsUp(sc, 2, 60000) - val rdd = sc.parallelize(0 until 1000, 10).map(i => (i, 1)).reduceByKey(_ + _) + val rdd = createRDD(sc) rdd.count() rdd.count() @@ -92,4 +93,16 @@ class ExternalShuffleServiceSuite extends ShuffleSuite with BeforeAndAfterAll { } e.getMessage should include ("Fetch failure will not retry stage due to testing config") } + + test("using external shuffle service") { + val createRDD = (sc: SparkContext) => + sc.parallelize(0 until 1000, 10).map(i => (i, 1)).reduceByKey(_ + _) + checkResultWithShuffleService(createRDD) + } + + test("using external shuffle service for indeterminate rdd") { + val createIndeterminateRDD = (sc: SparkContext) => + sc.parallelize(0 until 1000, 10).repartition(11).repartition(12) + checkResultWithShuffleService(createIndeterminateRDD) + } } diff --git a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala index d86975964b55..5925f43b6b3e 100644 --- a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala @@ -329,4 +329,53 @@ class MapOutputTrackerSuite extends SparkFunSuite { rpcEnv.shutdown() } + test("set and get indeterminate attempt id") { + val hostname = "localhost" + val rpcEnv = createRpcEnv("spark", hostname, 0, new SecurityManager(conf)) + + val masterTracker = newTrackerMaster() + masterTracker.trackerEndpoint = rpcEnv.setupEndpoint(MapOutputTracker.ENDPOINT_NAME, + new MapOutputTrackerMasterEndpoint(rpcEnv, masterTracker, conf)) + + val slaveRpcEnv = createRpcEnv("spark-slave", hostname, 0, new SecurityManager(conf)) + val slaveTracker = new MapOutputTrackerWorker(conf) + slaveTracker.trackerEndpoint = + slaveRpcEnv.setupEndpointRef(rpcEnv.address, MapOutputTracker.ENDPOINT_NAME) + + // Register shuffle and indeterminate shuffle Id. + masterTracker.registerShuffle(0, 2) + masterTracker.registerIndeterminateShuffle(0, 0) + slaveTracker.updateEpoch(masterTracker.getEpoch) + masterTracker.registerMapOutput(0, 0, MapStatus( + BlockManagerId("a", "hostA", 1000), Array(1000L))) + masterTracker.registerMapOutput(0, 1, MapStatus( + BlockManagerId("b", "hostB", 1000), Array(1000L))) + + // IndeterminateAttemptIds will be updated after call `getMapSizesByExecutorId` + slaveTracker.getMapSizesByExecutorId(0, 0, 1, needFetchIndeterminateAttemptId = true) + assert(slaveTracker.indeterminateAttemptIds(0).get == 0) + + // This is expected to fail because register same shuffleId with different attemptId + // is not allowed. + val error = intercept[IllegalArgumentException] { + masterTracker.registerIndeterminateShuffle(0, 1) + } + assert(error.getMessage.contains("The indeterminate stage should whole stage rerun")) + + // New attemptId is allowed after unregister old one. + masterTracker.unregisterIndeterminateShuffle(0) + masterTracker.registerIndeterminateShuffle(0, 1) + masterTracker.incrementEpoch() + slaveTracker.updateEpoch(masterTracker.getEpoch) + + // Update worker side indeterminateAttemptIds + slaveTracker.getMapSizesByExecutorId(0, 0, 1, needFetchIndeterminateAttemptId = true) + assert(slaveTracker.indeterminateAttemptIds(0).get == 1) + + masterTracker.stop() + slaveTracker.stop() + rpcEnv.shutdown() + slaveRpcEnv.shutdown() + } + } diff --git a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala index 8b1084a8edc7..b187d7138218 100644 --- a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala +++ b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala @@ -412,6 +412,28 @@ abstract class ShuffleSuite extends SparkFunSuite with Matchers with LocalSparkC manager.unregisterShuffle(0) } + + test("[SPARK-25341] check local shuffle with stage attempt id for indeterminate map stage") { + sc = new SparkContext("local", "test", conf.clone()) + // Repeated repartition operation will got an indeterminate map stage of shuffle 0. + val rdd = sc.parallelize(1 to 10, 2).repartition(4).repartition(5) + + // Cannot find one of the local shuffle blocks. + val dataFile = sc.env.blockManager.diskBlockManager.getFile( + new ShuffleDataBlockId(0, 0, 0, Some(0))) + val indexFile = sc.env.blockManager.diskBlockManager.getFile( + new ShuffleIndexBlockId(0, 0, 0, Some(0))) + assert(!dataFile.exists() && !indexFile.exists()) + + rdd.count() + + // Can find one of the local shuffle blocks. + val dataExistsFile = sc.env.blockManager.diskBlockManager.getFile( + new ShuffleDataBlockId(0, 0, 0, Some(0))) + val indexExistsFile = sc.env.blockManager.diskBlockManager.getFile( + new ShuffleIndexBlockId(0, 0, 0, Some(0))) + assert(dataExistsFile.exists() && indexExistsFile.exists()) + } } /** 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 21e85f441a5a..ff8badeda9ec 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -2700,7 +2700,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") { + test("SPARK-25341: retry all the succeeding stages when the map stage is indeterminate") { val shuffleMapRdd1 = new MyRDD(sc, 2, Nil, indeterminate = true) val shuffleDep1 = new ShuffleDependency(shuffleMapRdd1, new HashPartitioner(2)) @@ -2779,6 +2779,68 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi 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 + + 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)) + + // Finish the first 3 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() + } + private def assertResultStageFailToRollback(mapRdd: MyRDD): Unit = { val shuffleDep = new ShuffleDependency(mapRdd, new HashPartitioner(2)) val shuffleId = shuffleDep.shuffleId 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 27bb06b4e063..363cd30fd53f 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 @@ -58,10 +58,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" + private def testWithIndexShuffleBlockResolver( + shuffleId: Int, mapId: Int, idxName: String, attemptId: Option[Int]): Unit = { val resolver = new IndexShuffleBlockResolver(conf, blockManager) val lengths = Array[Long](10, 0, 20) val dataTmp = File.createTempFile("shuffle", null, tempDir) @@ -71,10 +69,10 @@ class IndexShuffleBlockResolverSuite extends SparkFunSuite with BeforeAndAfterEa } { out.close() } - resolver.writeIndexFileAndCommit(shuffleId, mapId, lengths, dataTmp) + resolver.writeIndexFileAndCommit(shuffleId, mapId, lengths, dataTmp, attemptId) val indexFile = new File(tempDir.getAbsolutePath, idxName) - val dataFile = resolver.getDataFile(shuffleId, mapId) + val dataFile = resolver.getDataFile(shuffleId, mapId, attemptId) assert(indexFile.exists()) assert(indexFile.length() === (lengths.length + 1) * 8) @@ -91,7 +89,7 @@ class IndexShuffleBlockResolverSuite extends SparkFunSuite with BeforeAndAfterEa } { out2.close() } - resolver.writeIndexFileAndCommit(shuffleId, mapId, lengths2, dataTmp2) + resolver.writeIndexFileAndCommit(shuffleId, mapId, lengths2, dataTmp2, attemptId) assert(indexFile.length() === (lengths.length + 1) * 8) assert(lengths2.toSeq === lengths.toSeq) @@ -130,7 +128,7 @@ class IndexShuffleBlockResolverSuite extends SparkFunSuite with BeforeAndAfterEa } { out3.close() } - resolver.writeIndexFileAndCommit(shuffleId, mapId, lengths3, dataTmp3) + resolver.writeIndexFileAndCommit(shuffleId, mapId, lengths3, dataTmp3, attemptId) assert(indexFile.length() === (lengths3.length + 1) * 8) assert(lengths3.toSeq != lengths.toSeq) assert(dataFile.exists()) @@ -155,4 +153,19 @@ class IndexShuffleBlockResolverSuite extends SparkFunSuite with BeforeAndAfterEa indexIn2.close() } } + + test("commit shuffle files multiple times") { + val shuffleId = 1 + val mapId = 2 + val idxName = s"shuffle_${shuffleId}_${mapId}_0.index" + testWithIndexShuffleBlockResolver(shuffleId, mapId, idxName, None) + } + + test("commit shuffle files with attempt id multiple times") { + val shuffleId = 1 + val mapId = 2 + val attemptId = Some(1) + val idxName = s"shuffle_${shuffleId}_${mapId}_0_1.index" + testWithIndexShuffleBlockResolver(shuffleId, mapId, idxName, attemptId) + } } From 4221f5014b540685f725c65edf17bd421b2dbbef Mon Sep 17 00:00:00 2001 From: Yuanjian Li Date: Thu, 14 Mar 2019 11:17:01 +0800 Subject: [PATCH 07/16] fix after rebase --- .../spark/shuffle/sort/BypassMergeSortShuffleWriter.java | 2 ++ .../org/apache/spark/shuffle/sort/SortShuffleManager.scala | 1 + .../scala/org/apache/spark/ExternalShuffleServiceSuite.scala | 2 +- .../shuffle/sort/BypassMergeSortShuffleWriterSuite.scala | 4 ++++ 4 files changed, 8 insertions(+), 1 deletion(-) 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 9d35963e14b0..9c99105182b4 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 @@ -38,6 +38,7 @@ import org.apache.spark.Partitioner; import org.apache.spark.ShuffleDependency; import org.apache.spark.SparkConf; +import org.apache.spark.TaskContext; import org.apache.spark.scheduler.MapStatus; import org.apache.spark.scheduler.MapStatus$; import org.apache.spark.serializer.Serializer; @@ -103,6 +104,7 @@ final class BypassMergeSortShuffleWriter extends ShuffleWriter { IndexShuffleBlockResolver shuffleBlockResolver, BypassMergeSortShuffleHandle handle, int mapId, + TaskContext taskContext, SparkConf conf, ShuffleWriteMetricsReporter writeMetrics) { // Use getSizeAsKb (not bytes) to maintain backwards compatibility if no units are provided 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 c8258aceec85..db6cd6f5281b 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 @@ -150,6 +150,7 @@ private[spark] class SortShuffleManager(conf: SparkConf) extends ShuffleManager shuffleBlockResolver.asInstanceOf[IndexShuffleBlockResolver], bypassMergeSortHandle, mapId, + context, env.conf, metrics) case other: BaseShuffleHandle[K @unchecked, V @unchecked, _] => diff --git a/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala b/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala index 6c0d7cefc37f..5e2e76c4937d 100644 --- a/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala +++ b/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala @@ -24,8 +24,8 @@ import org.apache.spark.network.TransportContext import org.apache.spark.network.netty.SparkTransportConf import org.apache.spark.network.server.TransportServer import org.apache.spark.network.shuffle.{ExternalShuffleBlockHandler, ExternalShuffleClient} -import org.apache.spark.util.Utils import org.apache.spark.rdd.RDD +import org.apache.spark.util.Utils /** * This suite creates an external shuffle server and routes all shuffle fetches through it. 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 57a71b007264..58b100ffd713 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 @@ -127,6 +127,7 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte blockResolver, shuffleHandle, 0, // MapId + taskContext, conf, taskContext.taskMetrics().shuffleWriteMetrics ) @@ -151,6 +152,7 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte blockResolver, shuffleHandle, 0, // MapId + taskContext, conf, taskContext.taskMetrics().shuffleWriteMetrics ) @@ -186,6 +188,7 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte blockResolver, shuffleHandle, 0, // MapId + taskContext, conf, taskContext.taskMetrics().shuffleWriteMetrics ) @@ -208,6 +211,7 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte blockResolver, shuffleHandle, 0, // MapId + taskContext, conf, taskContext.taskMetrics().shuffleWriteMetrics ) From a9921b18c763a14c8dcddc293c2d015cf1d434c5 Mon Sep 17 00:00:00 2001 From: Yuanjian Li Date: Fri, 15 Mar 2019 23:17:29 +0800 Subject: [PATCH 08/16] Bug fix after SPARK-25250 --- pom.xml | 1 - 1 file changed, 1 deletion(-) diff --git a/pom.xml b/pom.xml index 31d2948ec601..5879a76477dd 100644 --- a/pom.xml +++ b/pom.xml @@ -2149,7 +2149,6 @@ net.alchim31.maven scala-maven-plugin - 3.4.4 From 44d576c96cde106b94f33869f7394e16f931d782 Mon Sep 17 00:00:00 2001 From: Yuanjian Li Date: Thu, 28 Mar 2019 17:11:37 +0800 Subject: [PATCH 09/16] extend all shuffle block id, not only the indeterminate ones. --- .../shuffle/ExternalShuffleBlockHandler.java | 10 +- .../shuffle/ExternalShuffleBlockResolver.java | 30 ++--- .../network/sasl/SaslIntegrationSuite.java | 2 +- .../ExternalShuffleBlockHandlerSuite.java | 10 +- .../ExternalShuffleBlockResolverSuite.java | 36 +----- .../shuffle/ExternalShuffleCleanupSuite.java | 2 +- .../ExternalShuffleIntegrationSuite.java | 51 +++----- .../shuffle/NonShuffleFilesCleanupSuite.java | 2 +- .../shuffle/TestShuffleDataContext.java | 6 +- .../sort/BypassMergeSortShuffleWriter.java | 16 +-- .../shuffle/sort/UnsafeShuffleWriter.java | 11 +- .../org/apache/spark/BarrierTaskContext.scala | 7 -- .../scala/org/apache/spark/Dependency.scala | 5 +- .../org/apache/spark/MapOutputTracker.scala | 111 ++---------------- .../scala/org/apache/spark/TaskContext.scala | 9 -- .../org/apache/spark/TaskContextImpl.scala | 12 +- .../apache/spark/scheduler/DAGScheduler.scala | 10 +- .../apache/spark/scheduler/MapStatus.scala | 44 +++++-- .../spark/scheduler/ShuffleMapStage.scala | 1 - .../spark/scheduler/ShuffleMapTask.scala | 5 +- .../org/apache/spark/scheduler/Task.scala | 6 +- .../org/apache/spark/scheduler/TaskSet.scala | 3 +- .../shuffle/BlockStoreShuffleReader.scala | 4 +- .../shuffle/IndexShuffleBlockResolver.scala | 20 ++-- .../shuffle/sort/SortShuffleManager.scala | 20 ++-- .../shuffle/sort/SortShuffleWriter.scala | 10 +- .../org/apache/spark/storage/BlockId.scala | 56 +++------ .../sort/UnsafeShuffleWriterSuite.java | 4 +- .../apache/spark/MapOutputTrackerSuite.scala | 51 +------- .../scala/org/apache/spark/ShuffleSuite.scala | 8 +- .../spark/scheduler/MapStatusSuite.scala | 6 +- .../serializer/KryoSerializerSuite.scala | 3 +- .../BypassMergeSortShuffleWriterSuite.scala | 15 ++- .../sort/IndexShuffleBlockResolverSuite.scala | 8 +- .../apache/spark/storage/BlockIdSuite.scala | 30 ++--- project/MimaExcludes.scala | 2 - 36 files changed, 200 insertions(+), 426 deletions(-) diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java index 648aa94e928a..4a0dc09e8b2a 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java @@ -218,7 +218,7 @@ private class ManagedBufferIterator implements Iterator { private final int shuffleId; // An array containing mapId and reduceId pairs. private final int[] mapIdAndReduceIds; - private final int indeterminateAttemptId; + private final int stageAttemptId; ManagedBufferIterator(String appId, String execId, String[] blockIds) { this.appId = appId; @@ -227,8 +227,6 @@ private class ManagedBufferIterator implements Iterator { checkBlockId(blockId0Parts, blockIds, 0); this.shuffleId = Integer.parseInt(blockId0Parts[1]); mapIdAndReduceIds = new int[2 * blockIds.length]; - this.indeterminateAttemptId = - (blockId0Parts.length == 5) ? Integer.parseInt(blockId0Parts[4]) : -1; for (int i = 0; i < blockIds.length; i++) { String[] blockIdParts = blockIds[i].split("_"); checkBlockId(blockIdParts, blockIds, i); @@ -239,11 +237,11 @@ private class ManagedBufferIterator implements Iterator { mapIdAndReduceIds[2 * i] = Integer.parseInt(blockIdParts[2]); mapIdAndReduceIds[2 * i + 1] = Integer.parseInt(blockIdParts[3]); } + this.stageAttemptId = Integer.parseInt(blockId0Parts[4]); } private void checkBlockId(String[] blockIdParts, String[] wholeBlockId, int index) { - if ((blockIdParts.length != 4 && blockIdParts.length != 5) - || !blockIdParts[0].equals("shuffle")) { + if (blockIdParts.length != 5 || !blockIdParts[0].equals("shuffle")) { throw new IllegalArgumentException( "Unexpected shuffle block id format: " + wholeBlockId[index]); } @@ -257,7 +255,7 @@ public boolean hasNext() { @Override public ManagedBuffer next() { final ManagedBuffer block = blockManager.getBlockData(appId, execId, shuffleId, - mapIdAndReduceIds[index], mapIdAndReduceIds[index + 1], indeterminateAttemptId); + mapIdAndReduceIds[index], mapIdAndReduceIds[index + 1], stageAttemptId); index += 2; metrics.blockTransferRateBytes.mark(block != null ? block.size() : 0); return block; 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 9ef2044fb03f..7823625f869c 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 @@ -161,18 +161,6 @@ public void registerExecutor( executors.put(fullId, executorInfo); } - /** - * Overload getBlockData with setting indeterminateRetryId to an invalid value of -1. - */ - public ManagedBuffer getBlockData( - String appId, - String execId, - int shuffleId, - int mapId, - int reduceId) { - return getBlockData(appId, execId, shuffleId, mapId, reduceId, -1); - } - /** * Obtains a FileSegmentManagedBuffer from (shuffleId, mapId, reduceId). We make assumptions * about how the hash and sort based shuffles store their data. @@ -183,14 +171,14 @@ public ManagedBuffer getBlockData( int shuffleId, int mapId, int reduceId, - int indeterminateRetryId) { + int stageAttemptId) { 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, indeterminateRetryId); + executor, shuffleId, mapId, reduceId, stageAttemptId); } /** @@ -289,17 +277,15 @@ public boolean accept(File dir, String name) { } /** - * 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_MapId_0_StageAttemptId.index" + * into a data file called "shuffle_ShuffleId_MapId_0_StageAttemptId.data". This logic is from + * IndexShuffleBlockResolver, and the block id format is from ShuffleDataBlockId and + * ShuffleIndexBlockId. */ private ManagedBuffer getSortBasedShuffleBlockData( ExecutorShuffleInfo executor, int shuffleId, - int mapId, int reduceId, int indeterminateRetryId) { - String baseFileName = "shuffle_" + shuffleId + "_" + mapId + "_0"; - if (indeterminateRetryId != -1) { - baseFileName = baseFileName + "_" + indeterminateRetryId; - } + int mapId, int reduceId, int stageAttemptId) { + String baseFileName = "shuffle_" + shuffleId + "_" + mapId + "_0" + "_" + stageAttemptId; File indexFile = getFile(executor.localDirs, executor.subDirsPerLocalDir, baseFileName + ".index"); diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/sasl/SaslIntegrationSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/sasl/SaslIntegrationSuite.java index 57c1c5e7722c..913f6a7c451f 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/sasl/SaslIntegrationSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/sasl/SaslIntegrationSuite.java @@ -201,7 +201,7 @@ public void onBlockFetchFailure(String blockId, Throwable t) { } }; - String[] blockIds = { "shuffle_0_1_2", "shuffle_0_3_4" }; + String[] blockIds = { "shuffle_0_1_2_0", "shuffle_0_3_4_0" }; OneForOneBlockFetcher fetcher = new OneForOneBlockFetcher(client1, "app-2", "0", blockIds, listener, conf); fetcher.start(); diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandlerSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandlerSuite.java index 63344aed63f8..2c426bf584e3 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandlerSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandlerSuite.java @@ -85,10 +85,10 @@ public void testOpenShuffleBlocks() { ManagedBuffer block0Marker = new NioManagedBuffer(ByteBuffer.wrap(new byte[3])); ManagedBuffer block1Marker = new NioManagedBuffer(ByteBuffer.wrap(new byte[7])); - when(blockResolver.getBlockData("app0", "exec1", 0, 0, 0, -1)).thenReturn(block0Marker); - when(blockResolver.getBlockData("app0", "exec1", 0, 0, 1, -1)).thenReturn(block1Marker); + when(blockResolver.getBlockData("app0", "exec1", 0, 0, 0, 0)).thenReturn(block0Marker); + when(blockResolver.getBlockData("app0", "exec1", 0, 0, 1, 0)).thenReturn(block1Marker); ByteBuffer openBlocks = new OpenBlocks("app0", "exec1", - new String[] { "shuffle_0_0_0", "shuffle_0_0_1" }) + new String[] { "shuffle_0_0_0_0", "shuffle_0_0_1_0" }) .toByteBuffer(); handler.receive(client, openBlocks, callback); @@ -109,8 +109,8 @@ public void testOpenShuffleBlocks() { assertEquals(block0Marker, buffers.next()); assertEquals(block1Marker, buffers.next()); assertFalse(buffers.hasNext()); - verify(blockResolver, times(1)).getBlockData("app0", "exec1", 0, 0, 0, -1); - verify(blockResolver, times(1)).getBlockData("app0", "exec1", 0, 0, 1, -1); + verify(blockResolver, times(1)).getBlockData("app0", "exec1", 0, 0, 0, 0); + verify(blockResolver, times(1)).getBlockData("app0", "exec1", 0, 0, 1, 0); // Verify open block request latency metrics Timer openBlockRequestLatencyMillis = (Timer) ((ExternalShuffleBlockHandler) handler) diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolverSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolverSuite.java index 3b7cee2b0145..b18c353ee02c 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolverSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolverSuite.java @@ -53,10 +53,7 @@ public static void beforeAll() throws IOException { // Write some sort data. dataContext.insertSortShuffleData(0, 0, new byte[][] { sortBlock0.getBytes(StandardCharsets.UTF_8), - sortBlock1.getBytes(StandardCharsets.UTF_8)}, false); - dataContext.insertSortShuffleData(0, 0, new byte[][] { - sortBlock0.getBytes(StandardCharsets.UTF_8), - sortBlock1.getBytes(StandardCharsets.UTF_8)}, true); + sortBlock1.getBytes(StandardCharsets.UTF_8)}); } @AfterClass @@ -69,7 +66,7 @@ public void testBadRequests() throws IOException { ExternalShuffleBlockResolver resolver = new ExternalShuffleBlockResolver(conf, null); // Unregistered executor try { - resolver.getBlockData("app0", "exec1", 1, 1, 0); + resolver.getBlockData("app0", "exec1", 1, 1, 0, 0); fail("Should have failed"); } catch (RuntimeException e) { assertTrue("Bad error message: " + e, e.getMessage().contains("not registered")); @@ -78,7 +75,7 @@ public void testBadRequests() throws IOException { // Invalid shuffle manager try { resolver.registerExecutor("app0", "exec2", dataContext.createExecutorInfo("foobar")); - resolver.getBlockData("app0", "exec2", 1, 1, 0); + resolver.getBlockData("app0", "exec2", 1, 1, 0, 0); fail("Should have failed"); } catch (UnsupportedOperationException e) { // pass @@ -88,7 +85,7 @@ public void testBadRequests() throws IOException { resolver.registerExecutor("app0", "exec3", dataContext.createExecutorInfo(SORT_MANAGER)); try { - resolver.getBlockData("app0", "exec3", 1, 1, 0); + resolver.getBlockData("app0", "exec3", 1, 1, 0, 0); fail("Should have failed"); } catch (Exception e) { // pass @@ -101,38 +98,17 @@ public void testSortShuffleBlocks() throws IOException { resolver.registerExecutor("app0", "exec0", dataContext.createExecutorInfo(SORT_MANAGER)); - try (InputStream block0Stream = resolver.getBlockData( - "app0", "exec0", 0, 0, 0).createInputStream()) { - String block0 = - CharStreams.toString(new InputStreamReader(block0Stream, StandardCharsets.UTF_8)); - assertEquals(sortBlock0, block0); - } - - try (InputStream block1Stream = resolver.getBlockData( - "app0", "exec0", 0, 0, 1).createInputStream()) { - String block1 = - CharStreams.toString(new InputStreamReader(block1Stream, StandardCharsets.UTF_8)); - assertEquals(sortBlock1, block1); - } - } - - @Test - public void testExtendedSortShuffleBlocks() throws IOException { - ExternalShuffleBlockResolver resolver = new ExternalShuffleBlockResolver(conf, null); - resolver.registerExecutor("app0", "exec0", - dataContext.createExecutorInfo(SORT_MANAGER)); - try (InputStream block0Stream = resolver.getBlockData( "app0", "exec0", 0, 0, 0, 0).createInputStream()) { String block0 = - CharStreams.toString(new InputStreamReader(block0Stream, StandardCharsets.UTF_8)); + CharStreams.toString(new InputStreamReader(block0Stream, StandardCharsets.UTF_8)); assertEquals(sortBlock0, block0); } try (InputStream block1Stream = resolver.getBlockData( "app0", "exec0", 0, 0, 1, 0).createInputStream()) { String block1 = - CharStreams.toString(new InputStreamReader(block1Stream, StandardCharsets.UTF_8)); + CharStreams.toString(new InputStreamReader(block1Stream, StandardCharsets.UTF_8)); assertEquals(sortBlock1, block1); } } diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleCleanupSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleCleanupSuite.java index 259cc2e04b7d..47c087088a8a 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleCleanupSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleCleanupSuite.java @@ -142,7 +142,7 @@ private static TestShuffleDataContext createSomeData() throws IOException { dataContext.create(); dataContext.insertSortShuffleData(rand.nextInt(1000), rand.nextInt(1000), new byte[][] { "ABC".getBytes(StandardCharsets.UTF_8), - "DEF".getBytes(StandardCharsets.UTF_8)}, false); + "DEF".getBytes(StandardCharsets.UTF_8)}); return dataContext; } } diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java index 98418cf86ea7..4e490422c366 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java @@ -84,8 +84,7 @@ public static void beforeAll() throws IOException { dataContext0 = new TestShuffleDataContext(2, 5); dataContext0.create(); - dataContext0.insertSortShuffleData(0, 0, exec0Blocks, false); - dataContext0.insertSortShuffleData(0, 0, exec0Blocks, true); + dataContext0.insertSortShuffleData(0, 0, exec0Blocks); conf = new TransportConf("shuffle", MapConfigProvider.EMPTY); handler = new ExternalShuffleBlockHandler(conf, null); @@ -172,28 +171,6 @@ public void onBlockFetchFailure(String blockId, Throwable exception) { @Test public void testFetchOneSort() throws Exception { - registerExecutor("exec-0", dataContext0.createExecutorInfo(SORT_MANAGER)); - FetchResult exec0Fetch = fetchBlocks("exec-0", new String[] { "shuffle_0_0_0" }); - assertEquals(Sets.newHashSet("shuffle_0_0_0"), exec0Fetch.successBlocks); - assertTrue(exec0Fetch.failedBlocks.isEmpty()); - assertBufferListsEqual(exec0Fetch.buffers, Arrays.asList(exec0Blocks[0])); - exec0Fetch.releaseBuffers(); - } - - @Test - public void testFetchThreeSort() throws Exception { - registerExecutor("exec-0", dataContext0.createExecutorInfo(SORT_MANAGER)); - FetchResult exec0Fetch = fetchBlocks("exec-0", - new String[] { "shuffle_0_0_0", "shuffle_0_0_1", "shuffle_0_0_2" }); - assertEquals(Sets.newHashSet("shuffle_0_0_0", "shuffle_0_0_1", "shuffle_0_0_2"), - exec0Fetch.successBlocks); - assertTrue(exec0Fetch.failedBlocks.isEmpty()); - assertBufferListsEqual(exec0Fetch.buffers, Arrays.asList(exec0Blocks)); - exec0Fetch.releaseBuffers(); - } - - @Test - public void testFetchOneExtendedSort() throws Exception { registerExecutor("exec-0", dataContext0.createExecutorInfo(SORT_MANAGER)); FetchResult exec0Fetch = fetchBlocks("exec-0", new String[] { "shuffle_0_0_0_0" }); assertEquals(Sets.newHashSet("shuffle_0_0_0_0"), exec0Fetch.successBlocks); @@ -203,7 +180,7 @@ public void testFetchOneExtendedSort() throws Exception { } @Test - public void testFetchThreeExtendedSort() throws Exception { + public void testFetchThreeSort() throws Exception { registerExecutor("exec-0", dataContext0.createExecutorInfo(SORT_MANAGER)); FetchResult exec0Fetch = fetchBlocks("exec-0", new String[] { "shuffle_0_0_0_0", "shuffle_0_0_1_0", "shuffle_0_0_2_0" }); @@ -222,36 +199,36 @@ public void testRegisterInvalidExecutor() throws Exception { @Test public void testFetchWrongBlockId() throws Exception { registerExecutor("exec-1", dataContext0.createExecutorInfo(SORT_MANAGER)); - FetchResult execFetch = fetchBlocks("exec-1", new String[] { "rdd_1_0_0" }); + FetchResult execFetch = fetchBlocks("exec-1", new String[] { "rdd_1_0_0_0" }); assertTrue(execFetch.successBlocks.isEmpty()); - assertEquals(Sets.newHashSet("rdd_1_0_0"), execFetch.failedBlocks); + assertEquals(Sets.newHashSet("rdd_1_0_0_0"), execFetch.failedBlocks); } @Test public void testFetchNonexistent() throws Exception { registerExecutor("exec-0", dataContext0.createExecutorInfo(SORT_MANAGER)); FetchResult execFetch = fetchBlocks("exec-0", - new String[] { "shuffle_2_0_0" }); + new String[] { "shuffle_2_0_0_0" }); assertTrue(execFetch.successBlocks.isEmpty()); - assertEquals(Sets.newHashSet("shuffle_2_0_0"), execFetch.failedBlocks); + assertEquals(Sets.newHashSet("shuffle_2_0_0_0"), execFetch.failedBlocks); } @Test public void testFetchWrongExecutor() throws Exception { registerExecutor("exec-0", dataContext0.createExecutorInfo(SORT_MANAGER)); - FetchResult execFetch0 = fetchBlocks("exec-0", new String[] { "shuffle_0_0_0" /* right */}); - FetchResult execFetch1 = fetchBlocks("exec-0", new String[] { "shuffle_1_0_0" /* wrong */ }); - assertEquals(Sets.newHashSet("shuffle_0_0_0"), execFetch0.successBlocks); - assertEquals(Sets.newHashSet("shuffle_1_0_0"), execFetch1.failedBlocks); + FetchResult execFetch0 = fetchBlocks("exec-0", new String[] { "shuffle_0_0_0_0" /* right */}); + FetchResult execFetch1 = fetchBlocks("exec-0", new String[] { "shuffle_1_0_0_0" /* wrong */ }); + assertEquals(Sets.newHashSet("shuffle_0_0_0_0"), execFetch0.successBlocks); + assertEquals(Sets.newHashSet("shuffle_1_0_0_0"), execFetch1.failedBlocks); } @Test public void testFetchUnregisteredExecutor() throws Exception { registerExecutor("exec-0", dataContext0.createExecutorInfo(SORT_MANAGER)); FetchResult execFetch = fetchBlocks("exec-2", - new String[] { "shuffle_0_0_0", "shuffle_1_0_0" }); + new String[] { "shuffle_0_0_0_0", "shuffle_1_0_0_0" }); assertTrue(execFetch.successBlocks.isEmpty()); - assertEquals(Sets.newHashSet("shuffle_0_0_0", "shuffle_1_0_0"), execFetch.failedBlocks); + assertEquals(Sets.newHashSet("shuffle_0_0_0_0", "shuffle_1_0_0_0"), execFetch.failedBlocks); } @Test @@ -260,9 +237,9 @@ public void testFetchNoServer() throws Exception { new MapConfigProvider(ImmutableMap.of("spark.shuffle.io.maxRetries", "0"))); registerExecutor("exec-0", dataContext0.createExecutorInfo(SORT_MANAGER)); FetchResult execFetch = fetchBlocks("exec-0", - new String[]{"shuffle_1_0_0", "shuffle_1_0_1"}, clientConf, 1 /* port */); + new String[]{"shuffle_1_0_0_0", "shuffle_1_0_1_0"}, clientConf, 1 /* port */); assertTrue(execFetch.successBlocks.isEmpty()); - assertEquals(Sets.newHashSet("shuffle_1_0_0", "shuffle_1_0_1"), execFetch.failedBlocks); + assertEquals(Sets.newHashSet("shuffle_1_0_0_0", "shuffle_1_0_1_0"), execFetch.failedBlocks); } private static void registerExecutor(String executorId, ExecutorShuffleInfo executorInfo) diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/NonShuffleFilesCleanupSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/NonShuffleFilesCleanupSuite.java index 80b27010d903..d22f3ace4103 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/NonShuffleFilesCleanupSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/NonShuffleFilesCleanupSuite.java @@ -211,7 +211,7 @@ private static void createShuffleFiles(TestShuffleDataContext dataContext) throw Random rand = new Random(123); dataContext.insertSortShuffleData(rand.nextInt(1000), rand.nextInt(1000), new byte[][] { "ABC".getBytes(StandardCharsets.UTF_8), - "DEF".getBytes(StandardCharsets.UTF_8)}, false); + "DEF".getBytes(StandardCharsets.UTF_8)}); } private static void createNonShuffleFiles(TestShuffleDataContext dataContext) throws IOException { diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/TestShuffleDataContext.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/TestShuffleDataContext.java index f0904fbc42a1..84643683fa61 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/TestShuffleDataContext.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/TestShuffleDataContext.java @@ -68,10 +68,8 @@ public void cleanup() { } /** Creates reducer blocks in a sort-based data format within our local dirs. */ - public void insertSortShuffleData( - int shuffleId, int mapId, byte[][] blocks, boolean extendedBlockId) throws IOException { - String blockId = "shuffle_" + shuffleId + "_" + mapId + "_0"; - if (extendedBlockId) blockId += "_0"; + public void insertSortShuffleData(int shuffleId, int mapId, byte[][] blocks) throws IOException { + String blockId = "shuffle_" + shuffleId + "_" + mapId + "_0" + "_0"; OutputStream dataStream = null; DataOutputStream indexStream = null; 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 9c99105182b4..fd962cf566a2 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 @@ -84,7 +84,7 @@ final class BypassMergeSortShuffleWriter extends ShuffleWriter { private final int mapId; private final Serializer serializer; private final IndexShuffleBlockResolver shuffleBlockResolver; - private final Option indeterminateAttemptId; + private final int stageAttemptId; /** Array of file writers, one for each partition */ private DiskBlockObjectWriter[] partitionWriters; @@ -119,7 +119,7 @@ final class BypassMergeSortShuffleWriter extends ShuffleWriter { this.writeMetrics = writeMetrics; this.serializer = dep.serializer(); this.shuffleBlockResolver = shuffleBlockResolver; - this.indeterminateAttemptId = taskContext.getIndeterminateAttemptId(); + this.stageAttemptId = taskContext.stageAttemptNumber(); } @Override @@ -128,8 +128,9 @@ public void write(Iterator> records) throws IOException { if (!records.hasNext()) { partitionLengths = new long[numPartitions]; shuffleBlockResolver.writeIndexFileAndCommit( - shuffleId, mapId, partitionLengths, null, indeterminateAttemptId); - mapStatus = MapStatus$.MODULE$.apply(blockManager.shuffleServerId(), partitionLengths); + shuffleId, mapId, partitionLengths, null, stageAttemptId); + mapStatus = MapStatus$.MODULE$.apply( + blockManager.shuffleServerId(), partitionLengths, stageAttemptId); return; } final SerializerInstance serInstance = serializer.newInstance(); @@ -161,18 +162,19 @@ public void write(Iterator> records) throws IOException { } } - File output = shuffleBlockResolver.getDataFile(shuffleId, mapId, indeterminateAttemptId); + File output = shuffleBlockResolver.getDataFile(shuffleId, mapId, stageAttemptId); File tmp = Utils.tempFileWith(output); try { partitionLengths = writePartitionedFile(tmp); shuffleBlockResolver.writeIndexFileAndCommit( - shuffleId, mapId, partitionLengths, tmp, indeterminateAttemptId); + shuffleId, mapId, partitionLengths, tmp, stageAttemptId); } finally { if (tmp.exists() && !tmp.delete()) { logger.error("Error while deleting temp file {}", tmp.getAbsolutePath()); } } - mapStatus = MapStatus$.MODULE$.apply(blockManager.shuffleServerId(), partitionLengths); + mapStatus = MapStatus$.MODULE$.apply( + blockManager.shuffleServerId(), partitionLengths, stageAttemptId); } @VisibleForTesting 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 486c62c1a2c3..464978f7d48b 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 @@ -82,7 +82,7 @@ public class UnsafeShuffleWriter extends ShuffleWriter { private final int initialSortBufferSize; private final int inputBufferSizeInBytes; private final int outputBufferSizeInBytes; - private final Option indeterminateAttemptId; + private final int stageAttemptId; @Nullable private MapStatus mapStatus; @Nullable private ShuffleExternalSorter sorter; @@ -151,7 +151,7 @@ public UnsafeShuffleWriter( this.outputBufferSizeInBytes = (int) (long) sparkConf.get(package$.MODULE$.SHUFFLE_UNSAFE_FILE_OUTPUT_BUFFER_SIZE()) * 1024; open(); - this.indeterminateAttemptId = taskContext.getIndeterminateAttemptId(); + this.stageAttemptId = taskContext.stageAttemptNumber(); } private void updatePeakMemoryUsed() { @@ -234,7 +234,7 @@ void closeAndWriteOutput() throws IOException { sorter = null; final long[] partitionLengths; final File output = shuffleBlockResolver.getDataFile( - shuffleId, mapId, indeterminateAttemptId); + shuffleId, mapId, stageAttemptId); final File tmp = Utils.tempFileWith(output); try { try { @@ -247,13 +247,14 @@ void closeAndWriteOutput() throws IOException { } } shuffleBlockResolver.writeIndexFileAndCommit( - shuffleId, mapId, partitionLengths, tmp, indeterminateAttemptId); + shuffleId, mapId, partitionLengths, tmp, stageAttemptId); } finally { if (tmp.exists() && !tmp.delete()) { logger.error("Error while deleting temp file {}", tmp.getAbsolutePath()); } } - mapStatus = MapStatus$.MODULE$.apply(blockManager.shuffleServerId(), partitionLengths); + mapStatus = MapStatus$.MODULE$.apply( + blockManager.shuffleServerId(), partitionLengths, stageAttemptId); } @VisibleForTesting diff --git a/core/src/main/scala/org/apache/spark/BarrierTaskContext.scala b/core/src/main/scala/org/apache/spark/BarrierTaskContext.scala index f3abc5239f7e..a354f44a1be1 100644 --- a/core/src/main/scala/org/apache/spark/BarrierTaskContext.scala +++ b/core/src/main/scala/org/apache/spark/BarrierTaskContext.scala @@ -213,18 +213,11 @@ class BarrierTaskContext private[spark] ( taskContext.markTaskCompleted(error) } - override private[spark] def markAsIndeterminate(): Unit = { - taskContext.markAsIndeterminate() - } - override private[spark] def fetchFailed: Option[FetchFailedException] = { taskContext.fetchFailed } override private[spark] def getLocalProperties: Properties = taskContext.getLocalProperties - - override private[spark] def getIndeterminateAttemptId: Option[Int] = - taskContext.getIndeterminateAttemptId } @Experimental diff --git a/core/src/main/scala/org/apache/spark/Dependency.scala b/core/src/main/scala/org/apache/spark/Dependency.scala index 028047704cf8..fb051a8c0db8 100644 --- a/core/src/main/scala/org/apache/spark/Dependency.scala +++ b/core/src/main/scala/org/apache/spark/Dependency.scala @@ -20,7 +20,7 @@ package org.apache.spark import scala.reflect.ClassTag import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.rdd.{DeterministicLevel, RDD} +import org.apache.spark.rdd.RDD import org.apache.spark.serializer.Serializer import org.apache.spark.shuffle.{ShuffleHandle, ShuffleWriteProcessor} @@ -95,9 +95,6 @@ class ShuffleDependency[K: ClassTag, V: ClassTag, C: ClassTag]( val shuffleHandle: ShuffleHandle = _rdd.context.env.shuffleManager.registerShuffle( shuffleId, _rdd.partitions.length, this) - val mapSideIndeterminate: Boolean = - _rdd.outputDeterministicLevel == DeterministicLevel.INDETERMINATE - _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 a4072516f8ba..e2505a3abe6c 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -84,13 +84,6 @@ private class ShuffleStatus(numPartitions: Int) { */ private[this] var _numAvailableOutputs: Int = 0 - /** - * The bookkeeping for the indeterminate attempt id for corresponding ShuffleMapStage, most time - * its value is None cause the ShuffleMapStage's [[org.apache.spark.rdd.DeterministicLevel]] is - * DETERMINATE. - */ - private[this] var _indeterminateAttemptId: Option[Int] = None - /** * Register a map output. If there is already a registered location for the map output then it * will be replaced by the new location. @@ -214,43 +207,12 @@ private class ShuffleStatus(numPartitions: Int) { } cachedSerializedMapStatus = null } - - /** - * Set the _indeterminateAttemptId when the first task for rerunning indeterminate stage - * successfully ended, check the value with all following tasks cause the indeterminate stage - * should whole stage rerun. - */ - def setAndCheckIndeterminateAttemptId(stageAttemptId: Int): Unit = synchronized { - if (_indeterminateAttemptId.isEmpty) { - _indeterminateAttemptId = Some(stageAttemptId) - } else { - require(_indeterminateAttemptId.get == stageAttemptId, "The indeterminate stage should" + - s" whole stage rerun by attempt id: ${_indeterminateAttemptId.get}, but found a task" + - s" with different stageAttemptId $stageAttemptId.") - } - } - - /** - * The indeterminate attempt id for current shuffle status. - */ - def indeterminateAttemptId: Option[Int] = synchronized { - _indeterminateAttemptId - } - - /** - * Reset the indeterminate attempt id to None. - */ - def resetIndeterminateAttemptId(): Unit = synchronized { - _indeterminateAttemptId = None - } } private[spark] sealed trait MapOutputTrackerMessage private[spark] case class GetMapOutputStatuses(shuffleId: Int) extends MapOutputTrackerMessage private[spark] case object StopMapOutputTracker extends MapOutputTrackerMessage -private[spark] case class GetIndeterminateAttemptId(shuffleId: Int) - extends MapOutputTrackerMessage private[spark] case class GetMapOutputMessage(shuffleId: Int, context: RpcCallContext) @@ -271,11 +233,6 @@ private[spark] class MapOutputTrackerMasterEndpoint( logInfo("MapOutputTrackerMasterEndpoint stopped!") context.reply(true) stop() - - case GetIndeterminateAttemptId(shuffleId: Int) => - val hostPort = context.senderAddress.hostPort - logInfo(s"Asked to send indeterminate attempt id for shuffle $shuffleId to $hostPort") - context.reply(tracker.shuffleStatuses(shuffleId).indeterminateAttemptId) } } @@ -338,12 +295,8 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging * 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. */ - def getMapSizesByExecutorId( - shuffleId: Int, - startPartition: Int, - endPartition: Int, - needFetchIndeterminateAttemptId: Boolean = false) - : Iterator[(BlockManagerId, Seq[(BlockId, Long)])] + def getMapSizesByExecutorId(shuffleId: Int, startPartition: Int, endPartition: Int) + : Iterator[(BlockManagerId, Seq[(BlockId, Long)])] /** * Deletes map output status information for the specified shuffle stage. @@ -692,37 +645,19 @@ 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, - needFetchIndeterminateAttemptId: Boolean = false) - : Iterator[(BlockManagerId, Seq[(BlockId, Long)])] = { + def getMapSizesByExecutorId(shuffleId: Int, startPartition: Int, endPartition: Int) + : Iterator[(BlockManagerId, Seq[(BlockId, Long)])] = { logDebug(s"Fetching outputs for shuffle $shuffleId, partitions $startPartition-$endPartition") shuffleStatuses.get(shuffleId) match { case Some (shuffleStatus) => - val indeterminateAttemptId = if (needFetchIndeterminateAttemptId) { - shuffleStatuses(shuffleId).indeterminateAttemptId - } else { - None - } shuffleStatus.withMapStatuses { statuses => - MapOutputTracker.convertMapStatuses( - shuffleId, startPartition, endPartition, statuses, indeterminateAttemptId) + MapOutputTracker.convertMapStatuses(shuffleId, startPartition, endPartition, statuses) } case None => Iterator.empty } } - def registerIndeterminateShuffle(shuffleId: Int, stageAttemptId: Int): Unit = { - shuffleStatuses(shuffleId).setAndCheckIndeterminateAttemptId(stageAttemptId) - } - - def unregisterIndeterminateShuffle(shuffleId: Int): Unit = { - shuffleStatuses(shuffleId).resetIndeterminateAttemptId() - } - override def stop() { mapOutputRequests.offer(PoisonPill) threadpool.shutdown() @@ -743,29 +678,16 @@ private[spark] class MapOutputTrackerWorker(conf: SparkConf) extends MapOutputTr val mapStatuses: Map[Int, Array[MapStatus]] = new ConcurrentHashMap[Int, Array[MapStatus]]().asScala - val indeterminateAttemptIds: Map[Int, Option[Int]] = - new ConcurrentHashMap[Int, Option[Int]]().asScala - /** Remembers which map output locations are currently being fetched on an executor. */ private val fetching = new HashSet[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, - needFetchIndeterminateAttemptId: Boolean = false) + override def getMapSizesByExecutorId(shuffleId: Int, startPartition: Int, endPartition: Int) : Iterator[(BlockManagerId, Seq[(BlockId, Long)])] = { logDebug(s"Fetching outputs for shuffle $shuffleId, partitions $startPartition-$endPartition") val statuses = getStatuses(shuffleId) - val indeterminateAttemptId = if (needFetchIndeterminateAttemptId) { - getIndeterminateAttemptId(shuffleId) - } else { - None - } try { - MapOutputTracker.convertMapStatuses( - shuffleId, startPartition, endPartition, statuses, indeterminateAttemptId) + MapOutputTracker.convertMapStatuses(shuffleId, startPartition, endPartition, statuses) } catch { case e: MetadataFetchFailedException => // We experienced a fetch failure so our mapStatuses cache is outdated; clear it: @@ -774,19 +696,6 @@ private[spark] class MapOutputTrackerWorker(conf: SparkConf) extends MapOutputTr } } - private def getIndeterminateAttemptId(shuffleId: Int): Option[Int] = { - val id = indeterminateAttemptIds.get(shuffleId).orNull - if (id == null) { - val fetchedId = askTracker[Option[Int]](GetIndeterminateAttemptId(shuffleId)) - indeterminateAttemptIds.put(shuffleId, fetchedId) - logInfo(s"Getting the indeterminate attempt id $id for" + - s" shuffleId: $shuffleId, and update local cache.") - fetchedId - } else { - id - } - } - /** * Get or fetch the array of MapStatuses for a given shuffle ID. NOTE: clients MUST synchronize * on this array when reading it, because on the driver, we may be changing it in place. @@ -866,7 +775,6 @@ private[spark] class MapOutputTrackerWorker(conf: SparkConf) extends MapOutputTr logInfo("Updating epoch to " + newEpoch + " and clearing cache") epoch = newEpoch mapStatuses.clear() - indeterminateAttemptIds.clear() } } } @@ -963,8 +871,7 @@ private[spark] object MapOutputTracker extends Logging { shuffleId: Int, startPartition: Int, endPartition: Int, - statuses: Array[MapStatus], - indeterminateAttemptId: Option[Int]): Iterator[(BlockManagerId, Seq[(BlockId, Long)])] = { + 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) { @@ -977,7 +884,7 @@ private[spark] object MapOutputTracker extends Logging { val size = status.getSizeForBlock(part) if (size != 0) { splitsByAddress.getOrElseUpdate(status.location, ListBuffer()) += - ((ShuffleBlockId(shuffleId, mapId, part, indeterminateAttemptId), size)) + ((ShuffleBlockId(shuffleId, mapId, part, status.stageAttemptId), size)) } } } diff --git a/core/src/main/scala/org/apache/spark/TaskContext.scala b/core/src/main/scala/org/apache/spark/TaskContext.scala index 1ec471b1ac36..959f246f3f9f 100644 --- a/core/src/main/scala/org/apache/spark/TaskContext.scala +++ b/core/src/main/scala/org/apache/spark/TaskContext.scala @@ -223,18 +223,9 @@ abstract class TaskContext extends Serializable { /** Marks the task as completed and triggers the completion listeners. */ private[spark] def markTaskCompleted(error: Option[Throwable]): Unit - /** Marks the task as indeterminate task. */ - private[spark] def markAsIndeterminate(): Unit - /** Optionally returns the stored fetch failure in the task. */ private[spark] def fetchFailed: Option[FetchFailedException] /** Gets local properties set upstream in the driver. */ private[spark] def getLocalProperties: Properties - - /** - * Get the indeterminate stage attempt id, it will return None if the task is determinate. - * While the task is indeterminate it return the stage attempt number. - */ - private[spark] def getIndeterminateAttemptId: Option[Int] } diff --git a/core/src/main/scala/org/apache/spark/TaskContextImpl.scala b/core/src/main/scala/org/apache/spark/TaskContextImpl.scala index 39efb42c662c..76296c5d0abd 100644 --- a/core/src/main/scala/org/apache/spark/TaskContextImpl.scala +++ b/core/src/main/scala/org/apache/spark/TaskContextImpl.scala @@ -73,9 +73,6 @@ private[spark] class TaskContextImpl( // Throwable that caused the task to fail private var failure: Throwable = _ - // Whether the task is determinate. - private var determinate: Boolean = true - // If there was a fetch failure in the task, we store it here, to make sure user-code doesn't // hide the exception. See SPARK-19276 @volatile private var _fetchFailedException: Option[FetchFailedException] = None @@ -146,10 +143,6 @@ private[spark] class TaskContextImpl( reasonIfKilled = Some(reason) } - private[spark] override def markAsIndeterminate(): Unit = { - determinate = false - } - private[spark] override def killTaskIfInterrupted(): Unit = { val reason = reasonIfKilled if (reason.isDefined) { @@ -181,8 +174,5 @@ private[spark] class TaskContextImpl( private[spark] override def fetchFailed: Option[FetchFailedException] = _fetchFailedException - private[spark] override def getLocalProperties: Properties = localProperties - - private[spark] override def getIndeterminateAttemptId: Option[Int] = - if (!determinate) Some(stageAttemptNumber) else None + private[spark] override def getLocalProperties(): Properties = localProperties } 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 25a331c094a7..8b2461d6b1f9 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1208,8 +1208,7 @@ private[spark] class DAGScheduler( stage.pendingPartitions += id new ShuffleMapTask(stage.id, stage.latestInfo.attemptNumber, taskBinary, part, locs, properties, serializedTaskMetrics, Option(jobId), - Option(sc.applicationId), sc.applicationAttemptId, stage.rdd.isBarrier(), - stage.isIndeterminate) + Option(sc.applicationId), sc.applicationAttemptId, stage.rdd.isBarrier()) } case stage: ResultStage => @@ -1234,7 +1233,8 @@ private[spark] class DAGScheduler( logInfo(s"Submitting ${tasks.size} missing tasks from $stage (${stage.rdd}) (first 15 " + s"tasks are for partitions ${tasks.take(15).map(_.partitionId)})") taskScheduler.submitTasks(new TaskSet( - tasks.toArray, stage.id, stage.latestInfo.attemptNumber, jobId, properties)) + tasks.toArray, stage.id, stage.latestInfo.attemptNumber, jobId, properties, + stage.latestInfo.attemptNumber > 0 && stage.isIndeterminate)) } else { // Because we posted SparkListenerStageSubmitted earlier, we should mark // the stage as completed here in case there are no tasks to run @@ -1454,10 +1454,6 @@ private[spark] class DAGScheduler( // available. mapOutputTracker.registerMapOutput( shuffleStage.shuffleDep.shuffleId, smt.partitionId, status) - if (stage.isIndeterminate) { - mapOutputTracker.registerIndeterminateShuffle( - shuffleStage.shuffleDep.shuffleId, smt.stageAttemptId) - } } if (runningStages.contains(shuffleStage) && shuffleStage.pendingPartitions.isEmpty) { 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 64f0a060a247..a80db00564cf 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 attempt id of the stage which this task belong to. + */ + def stageAttemptId: Int } @@ -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], + stageAttemptId: Int = 0): MapStatus = { if (uncompressedSizes.length > minPartitionsToUseHighlyCompressMapStatus) { - HighlyCompressedMapStatus(loc, uncompressedSizes) + HighlyCompressedMapStatus(loc, uncompressedSizes, stageAttemptId) } else { - new CompressedMapStatus(loc, uncompressedSizes) + new CompressedMapStatus(loc, uncompressedSizes, stageAttemptId) } } @@ -100,16 +108,18 @@ 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 stageAtmpId the attempt id of the stage which this task belong to. */ private[spark] class CompressedMapStatus( private[this] var loc: BlockManagerId, - private[this] var compressedSizes: Array[Byte]) + private[this] var compressedSizes: Array[Byte], + private[this] var stageAtmpId: Int) extends MapStatus with Externalizable { - protected def this() = this(null, null.asInstanceOf[Array[Byte]]) // For deserialization only + protected def this() = this(null, null.asInstanceOf[Array[Byte]], 0) // For deserialization only - def this(loc: BlockManagerId, uncompressedSizes: Array[Long]) { - this(loc, uncompressedSizes.map(MapStatus.compressSize)) + def this(loc: BlockManagerId, uncompressedSizes: Array[Long], stageAttemptId: Int) { + this(loc, uncompressedSizes.map(MapStatus.compressSize), stageAttemptId) } override def location: BlockManagerId = loc @@ -122,6 +132,7 @@ private[spark] class CompressedMapStatus( loc.writeExternal(out) out.writeInt(compressedSizes.length) out.write(compressedSizes) + out.writeInt(stageAtmpId) } override def readExternal(in: ObjectInput): Unit = Utils.tryOrIOException { @@ -129,7 +140,10 @@ private[spark] class CompressedMapStatus( val len = in.readInt() compressedSizes = new Array[Byte](len) in.readFully(compressedSizes) + stageAtmpId = in.readInt() } + + override def stageAttemptId: Int = stageAtmpId } /** @@ -142,23 +156,27 @@ 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 stageAtmpId the attempt id of the stage which this task belong to. */ 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 stageAtmpId: Int) 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, "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, 0) // For deserialization only override def location: BlockManagerId = loc + override def stageAttemptId: Int = stageAtmpId + override def getSizeForBlock(reduceId: Int): Long = { assert(hugeBlockSizes != null) if (emptyBlocks.contains(reduceId)) { @@ -180,6 +198,7 @@ private[spark] class HighlyCompressedMapStatus private ( out.writeInt(kv._1) out.writeByte(kv._2) } + out.writeInt(stageAtmpId) } override def readExternal(in: ObjectInput): Unit = Utils.tryOrIOException { @@ -195,11 +214,14 @@ private[spark] class HighlyCompressedMapStatus private ( hugeBlockSizesImpl(block) = size } hugeBlockSizes = hugeBlockSizesImpl + stageAtmpId = in.readInt() } } private[spark] object HighlyCompressedMapStatus { - def apply(loc: BlockManagerId, uncompressedSizes: Array[Long]): HighlyCompressedMapStatus = { + def apply(loc: BlockManagerId, + uncompressedSizes: Array[Long], + stageAttemptId: Int): 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 +262,6 @@ private[spark] object HighlyCompressedMapStatus { emptyBlocks.trim() emptyBlocks.runOptimize() new HighlyCompressedMapStatus(loc, numNonEmptyBlocks, emptyBlocks, avgSize, - hugeBlockSizes) + hugeBlockSizes, stageAttemptId) } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapStage.scala b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapStage.scala index 62660fd46026..3bc0d302e500 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapStage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapStage.scala @@ -103,7 +103,6 @@ private[spark] class ShuffleMapStage( */ override def clearIntermediateState(): Unit = { if (isIndeterminate) { - mapOutputTrackerMaster.unregisterIndeterminateShuffle(shuffleDep.shuffleId) mapOutputTrackerMaster.unregisterAllMapOutput(shuffleDep.shuffleId) } } 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 43a693961fcb..189e35ee8311 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala @@ -63,10 +63,9 @@ private[spark] class ShuffleMapTask( jobId: Option[Int] = None, appId: Option[String] = None, appAttemptId: Option[String] = None, - isBarrier: Boolean = false, - isIndeterminate: Boolean = true) + isBarrier: Boolean = false) extends Task[MapStatus](stageId, stageAttemptId, partition.index, localProperties, - serializedTaskMetrics, jobId, appId, appAttemptId, isBarrier, isIndeterminate) + serializedTaskMetrics, jobId, appId, appAttemptId, isBarrier) with Logging { /** A constructor used only in test suites. This does not require passing in an RDD. */ diff --git a/core/src/main/scala/org/apache/spark/scheduler/Task.scala b/core/src/main/scala/org/apache/spark/scheduler/Task.scala index b11047e4c788..c17a9038ba14 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Task.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Task.scala @@ -51,7 +51,6 @@ import org.apache.spark.util._ * @param appAttemptId attempt id of the app this task belongs to * @param isBarrier whether this task belongs to a barrier stage. Spark must launch all the tasks * at the same time for a barrier stage. - * @param isIndeterminate whether this task is determinate. */ private[spark] abstract class Task[T]( val stageId: Int, @@ -64,8 +63,7 @@ private[spark] abstract class Task[T]( val jobId: Option[Int] = None, val appId: Option[String] = None, val appAttemptId: Option[String] = None, - val isBarrier: Boolean = false, - val isIndeterminate: Boolean = false) extends Serializable { + val isBarrier: Boolean = false) extends Serializable { @transient lazy val metrics: TaskMetrics = SparkEnv.get.closureSerializer.newInstance().deserialize(ByteBuffer.wrap(serializedTaskMetrics)) @@ -101,8 +99,6 @@ private[spark] abstract class Task[T]( taskContext } - if (isIndeterminate) context.markAsIndeterminate() - TaskContext.setTaskContext(context) taskThread = Thread.currentThread() diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSet.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSet.scala index 517c8991aed7..d13bfd331d96 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSet.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSet.scala @@ -28,7 +28,8 @@ private[spark] class TaskSet( val stageId: Int, val stageAttemptId: Int, val priority: Int, - val properties: Properties) { + val properties: Properties, + val retryIndeterminateStage: Boolean = false) { val id: String = stageId + "." + stageAttemptId override def toString: String = "TaskSet " + id 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 e1311401844b..c7843710413d 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala @@ -19,7 +19,6 @@ package org.apache.spark.shuffle import org.apache.spark._ import org.apache.spark.internal.{config, Logging} -import org.apache.spark.rdd.DeterministicLevel import org.apache.spark.serializer.SerializerManager import org.apache.spark.storage.{BlockManager, ShuffleBlockFetcherIterator} import org.apache.spark.util.CompletionIterator @@ -48,8 +47,7 @@ private[spark] class BlockStoreShuffleReader[K, C]( context, blockManager.shuffleClient, blockManager, - mapOutputTracker.getMapSizesByExecutorId( - handle.shuffleId, startPartition, endPartition, dep.mapSideIndeterminate), + mapOutputTracker.getMapSizesByExecutorId(handle.shuffleId, startPartition, endPartition), 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/IndexShuffleBlockResolver.scala b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala index 9ec8fd9d1c8a..d8b1df8477b1 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala @@ -54,7 +54,7 @@ private[spark] class IndexShuffleBlockResolver( def getDataFile( shuffleId: Int, mapId: Int, - stageAttemptId: Option[Int] = None): File = { + stageAttemptId: Int): File = { blockManager.diskBlockManager.getFile( ShuffleDataBlockId(shuffleId, mapId, NOOP_REDUCE_ID, stageAttemptId)) } @@ -62,7 +62,7 @@ private[spark] class IndexShuffleBlockResolver( private def getIndexFile( shuffleId: Int, mapId: Int, - stageAttemptId: Option[Int] = None): File = { + stageAttemptId: Int): File = { blockManager.diskBlockManager.getFile( ShuffleIndexBlockId(shuffleId, mapId, NOOP_REDUCE_ID, stageAttemptId)) } @@ -71,15 +71,15 @@ private[spark] class IndexShuffleBlockResolver( * Remove data file and index file that contain the output data from one map. */ def removeDataByMap( - shuffleId: Int, mapId: Int, indeterminateStageAttemptId: Option[Int]): Unit = { - var file = getDataFile(shuffleId, mapId, indeterminateStageAttemptId) + shuffleId: Int, mapId: Int, stageAttemptId: Int): Unit = { + var file = getDataFile(shuffleId, mapId, stageAttemptId) if (file.exists()) { if (!file.delete()) { logWarning(s"Error deleting data ${file.getPath()}") } } - file = getIndexFile(shuffleId, mapId, indeterminateStageAttemptId) + file = getIndexFile(shuffleId, mapId, stageAttemptId) if (file.exists()) { if (!file.delete()) { logWarning(s"Error deleting index ${file.getPath()}") @@ -147,11 +147,11 @@ private[spark] class IndexShuffleBlockResolver( mapId: Int, lengths: Array[Long], dataTmp: File, - indeterminateAttemptId: Option[Int] = None): Unit = { - val indexFile = getIndexFile(shuffleId, mapId, indeterminateAttemptId) + stageAttemptId: Int): Unit = { + val indexFile = getIndexFile(shuffleId, mapId, stageAttemptId) val indexTmp = Utils.tempFileWith(indexFile) try { - val dataFile = getDataFile(shuffleId, mapId, indeterminateAttemptId) + val dataFile = getDataFile(shuffleId, mapId, stageAttemptId) // There is only one IndexShuffleBlockResolver per executor, this synchronization make sure // the following check and rename are atomic. synchronized { @@ -203,7 +203,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, blockId.indeterminateAttemptId) + val indexFile = getIndexFile(blockId.shuffleId, blockId.mapId, blockId.stageAttemptId) // 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 @@ -225,7 +225,7 @@ private[spark] class IndexShuffleBlockResolver( } new FileSegmentManagedBuffer( transportConf, - getDataFile(blockId.shuffleId, blockId.mapId, blockId.indeterminateAttemptId), + getDataFile(blockId.shuffleId, blockId.mapId, blockId.stageAttemptId), 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 db6cd6f5281b..2085d44eb9c7 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.JavaConverters._ + import org.apache.spark._ import org.apache.spark.internal.Logging import org.apache.spark.shuffle._ @@ -75,10 +77,10 @@ private[spark] class SortShuffleManager(conf: SparkConf) extends ShuffleManager } /** - * A mapping from shuffle ids to the tuple of number of mappers producing output and - * indeterminate stage attempt id for those shuffles. + * A mapping from the tuple of shuffle ids and stage attempt ids to the number of mappers + * producing output for those shuffles. */ - private[this] val numMapsForShuffle = new ConcurrentHashMap[Int, (Int, Option[Int])]() + private[this] val numMapsForShuffle = new ConcurrentHashMap[(Int, Int), Int]().asScala override val shuffleBlockResolver = new IndexShuffleBlockResolver(conf) @@ -129,9 +131,8 @@ private[spark] class SortShuffleManager(conf: SparkConf) extends ShuffleManager context: TaskContext, metrics: ShuffleWriteMetricsReporter): ShuffleWriter[K, V] = { numMapsForShuffle.putIfAbsent( - handle.shuffleId, - (handle.asInstanceOf[BaseShuffleHandle[_, _, _]].numMaps, - context.getIndeterminateAttemptId)) + (handle.shuffleId, context.stageAttemptNumber()), + handle.asInstanceOf[BaseShuffleHandle[_, _, _]].numMaps) val env = SparkEnv.get handle match { case unsafeShuffleHandle: SerializedShuffleHandle[K @unchecked, V @unchecked] => @@ -160,10 +161,11 @@ private[spark] class SortShuffleManager(conf: SparkConf) extends ShuffleManager /** Remove a shuffle's metadata from the ShuffleManager. */ override def unregisterShuffle(shuffleId: Int): Boolean = { - Option(numMapsForShuffle.remove(shuffleId)).foreach { - case (numMaps, indeterminateAttemptId) => + numMapsForShuffle.filterKeys(_._1 == shuffleId).foreach { + case ((_, stageAttemptId), numMaps) => + numMapsForShuffle.remove((shuffleId, stageAttemptId)) (0 until numMaps).foreach { mapId => - shuffleBlockResolver.removeDataByMap(shuffleId, mapId, indeterminateAttemptId) + shuffleBlockResolver.removeDataByMap(shuffleId, mapId, stageAttemptId) } } 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 1061da7a3da2..bb6a4b160d73 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 @@ -47,7 +47,7 @@ private[spark] class SortShuffleWriter[K, V, C]( private val writeMetrics = context.taskMetrics().shuffleWriteMetrics - private val indeterminateAttemptId = context.getIndeterminateAttemptId + private val stageAttemptId = context.stageAttemptNumber() /** Write a bunch of records to this task's output */ override def write(records: Iterator[Product2[K, V]]): Unit = { @@ -66,18 +66,18 @@ 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 output = shuffleBlockResolver.getDataFile(dep.shuffleId, mapId, indeterminateAttemptId) + val output = shuffleBlockResolver.getDataFile(dep.shuffleId, mapId, stageAttemptId) val tmp = Utils.tempFileWith(output) try { val blockId = ShuffleBlockId( dep.shuffleId, mapId, IndexShuffleBlockResolver.NOOP_REDUCE_ID, - indeterminateAttemptId) + stageAttemptId) val partitionLengths = sorter.writePartitionedFile(blockId, tmp) shuffleBlockResolver.writeIndexFileAndCommit( - dep.shuffleId, mapId, partitionLengths, tmp, indeterminateAttemptId) - mapStatus = MapStatus(blockManager.shuffleServerId, partitionLengths) + dep.shuffleId, mapId, partitionLengths, tmp, stageAttemptId) + mapStatus = MapStatus(blockManager.shuffleServerId, partitionLengths, stageAttemptId) } finally { if (tmp.exists() && !tmp.delete()) { logError(s"Error while deleting temp file ${tmp.getAbsolutePath}") 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 8ce1e7e712f5..2fcf88f0c51f 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockId.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockId.scala @@ -58,12 +58,10 @@ case class ShuffleBlockId( shuffleId: Int, mapId: Int, reduceId: Int, - indeterminateAttemptId: Option[Int] = None) + stageAttemptId: Int = 0) extends BlockId { - override def name: String = { - val nameStr = "shuffle_" + shuffleId + "_" + mapId + "_" + reduceId - if (indeterminateAttemptId.isEmpty) nameStr else nameStr + "_" + indeterminateAttemptId.get - } + override def name: String = + "shuffle_" + shuffleId + "_" + mapId + "_" + reduceId + "_" + stageAttemptId } @DeveloperApi @@ -71,14 +69,10 @@ case class ShuffleDataBlockId( shuffleId: Int, mapId: Int, reduceId: Int, - indeterminateAttemptId: Option[Int] = None) + stageAttemptId: Int = 0) extends BlockId { - override def name: String = { - val nameStr = "shuffle_" + shuffleId + "_" + mapId + "_" + reduceId - val nameStrWithIndeterminateAttempt = - if (indeterminateAttemptId.isEmpty) nameStr else nameStr + "_" + indeterminateAttemptId.get - nameStrWithIndeterminateAttempt + ".data" - } + override def name: String = + "shuffle_" + shuffleId + "_" + mapId + "_" + reduceId + "_" + stageAttemptId + ".data" } @DeveloperApi @@ -86,14 +80,10 @@ case class ShuffleIndexBlockId( shuffleId: Int, mapId: Int, reduceId: Int, - indeterminateAttemptId: Option[Int] = None) + stageAttemptId: Int = 0) extends BlockId { - override def name: String = { - val nameStr = "shuffle_" + shuffleId + "_" + mapId + "_" + reduceId - val nameStrWithIndeterminateAttempt = - if (indeterminateAttemptId.isEmpty) nameStr else nameStr + "_" + indeterminateAttemptId.get - nameStrWithIndeterminateAttempt + ".index" - } + override def name: String = + "shuffle_" + shuffleId + "_" + mapId + "_" + reduceId + "_" + stageAttemptId + ".index" } @DeveloperApi @@ -133,13 +123,9 @@ class UnrecognizedBlockId(name: String) @DeveloperApi object BlockId { val RDD = "rdd_([0-9]+)_([0-9]+)".r - val SHUFFLE = "shuffle_([0-9]+)_([0-9]+)_([0-9]+)".r - val SHUFFLE_DATA = "shuffle_([0-9]+)_([0-9]+)_([0-9]+).data".r - val SHUFFLE_INDEX = "shuffle_([0-9]+)_([0-9]+)_([0-9]+).index".r - // the extend shuffle/data/index is only used when INDETERMINATE stage rerun - val EXTEND_SHUFFLE = "shuffle_([0-9]+)_([0-9]+)_([0-9]+)_([0-9]+)".r - val EXTEND_SHUFFLE_DATA = "shuffle_([0-9]+)_([0-9]+)_([0-9]+)_([0-9]+).data".r - val EXTEND_SHUFFLE_INDEX = "shuffle_([0-9]+)_([0-9]+)_([0-9]+)_([0-9]+).index".r + val SHUFFLE = "shuffle_([0-9]+)_([0-9]+)_([0-9]+)_([0-9]+)".r + val SHUFFLE_DATA = "shuffle_([0-9]+)_([0-9]+)_([0-9]+)_([0-9]+).data".r + val SHUFFLE_INDEX = "shuffle_([0-9]+)_([0-9]+)_([0-9]+)_([0-9]+).index".r val BROADCAST = "broadcast_([0-9]+)([_A-Za-z0-9]*)".r val TASKRESULT = "taskresult_([0-9]+)".r val STREAM = "input-([0-9]+)-([0-9]+)".r @@ -150,18 +136,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.toInt, reduceId.toInt) - case SHUFFLE_DATA(shuffleId, mapId, reduceId) => - ShuffleDataBlockId(shuffleId.toInt, mapId.toInt, reduceId.toInt) - case SHUFFLE_INDEX(shuffleId, mapId, reduceId) => - ShuffleIndexBlockId(shuffleId.toInt, mapId.toInt, reduceId.toInt) - case EXTEND_SHUFFLE(shuffleId, mapId, reduceId, stageAttemptId) => - ShuffleBlockId(shuffleId.toInt, mapId.toInt, reduceId.toInt, Some(stageAttemptId.toInt)) - case EXTEND_SHUFFLE_DATA(shuffleId, mapId, reduceId, stageAttemptId) => - ShuffleDataBlockId(shuffleId.toInt, mapId.toInt, reduceId.toInt, Some(stageAttemptId.toInt)) - case EXTEND_SHUFFLE_INDEX(shuffleId, mapId, reduceId, stageAttemptId) => - ShuffleIndexBlockId(shuffleId.toInt, mapId.toInt, reduceId.toInt, Some(stageAttemptId.toInt)) + case SHUFFLE(shuffleId, mapId, reduceId, stageAttemptId) => + ShuffleBlockId(shuffleId.toInt, mapId.toInt, reduceId.toInt, stageAttemptId.toInt) + case SHUFFLE_DATA(shuffleId, mapId, reduceId, stageAttemptId) => + ShuffleDataBlockId(shuffleId.toInt, mapId.toInt, reduceId.toInt, stageAttemptId.toInt) + case SHUFFLE_INDEX(shuffleId, mapId, reduceId, stageAttemptId) => + ShuffleIndexBlockId(shuffleId.toInt, mapId.toInt, reduceId.toInt, stageAttemptId.toInt) case BROADCAST(broadcastId, field) => BroadcastBlockId(broadcastId.toLong, field.stripPrefix("_")) case TASKRESULT(taskId) => 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 b6405b4bbbee..96260823d7f3 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 @@ -132,7 +132,7 @@ public void setUp() throws IOException { }); when(shuffleBlockResolver.getDataFile( - anyInt(), anyInt(), any())).thenReturn(mergedOutputFile); + anyInt(), anyInt(), anyInt())).thenReturn(mergedOutputFile); doAnswer(invocationOnMock -> { partitionSizesInMergedFile = (long[]) invocationOnMock.getArguments()[2]; File tmp = (File) invocationOnMock.getArguments()[3]; @@ -141,7 +141,7 @@ public void setUp() throws IOException { return null; }).when(shuffleBlockResolver) .writeIndexFileAndCommit( - anyInt(), anyInt(), any(long[].class), any(File.class), any(Option.class)); + anyInt(), anyInt(), any(long[].class), any(File.class), any(int.class)); when(diskBlockManager.createTempShuffleBlock()).thenAnswer(invocationOnMock -> { TempShuffleBlockId blockId = new TempShuffleBlockId(UUID.randomUUID()); diff --git a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala index 5925f43b6b3e..7cf6d3e431a0 100644 --- a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala @@ -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), 0)) } val senderAddress = RpcAddress("localhost", 12345) val rpcCallContext = mock(classOf[RpcCallContext]) @@ -329,53 +329,4 @@ class MapOutputTrackerSuite extends SparkFunSuite { rpcEnv.shutdown() } - test("set and get indeterminate attempt id") { - val hostname = "localhost" - val rpcEnv = createRpcEnv("spark", hostname, 0, new SecurityManager(conf)) - - val masterTracker = newTrackerMaster() - masterTracker.trackerEndpoint = rpcEnv.setupEndpoint(MapOutputTracker.ENDPOINT_NAME, - new MapOutputTrackerMasterEndpoint(rpcEnv, masterTracker, conf)) - - val slaveRpcEnv = createRpcEnv("spark-slave", hostname, 0, new SecurityManager(conf)) - val slaveTracker = new MapOutputTrackerWorker(conf) - slaveTracker.trackerEndpoint = - slaveRpcEnv.setupEndpointRef(rpcEnv.address, MapOutputTracker.ENDPOINT_NAME) - - // Register shuffle and indeterminate shuffle Id. - masterTracker.registerShuffle(0, 2) - masterTracker.registerIndeterminateShuffle(0, 0) - slaveTracker.updateEpoch(masterTracker.getEpoch) - masterTracker.registerMapOutput(0, 0, MapStatus( - BlockManagerId("a", "hostA", 1000), Array(1000L))) - masterTracker.registerMapOutput(0, 1, MapStatus( - BlockManagerId("b", "hostB", 1000), Array(1000L))) - - // IndeterminateAttemptIds will be updated after call `getMapSizesByExecutorId` - slaveTracker.getMapSizesByExecutorId(0, 0, 1, needFetchIndeterminateAttemptId = true) - assert(slaveTracker.indeterminateAttemptIds(0).get == 0) - - // This is expected to fail because register same shuffleId with different attemptId - // is not allowed. - val error = intercept[IllegalArgumentException] { - masterTracker.registerIndeterminateShuffle(0, 1) - } - assert(error.getMessage.contains("The indeterminate stage should whole stage rerun")) - - // New attemptId is allowed after unregister old one. - masterTracker.unregisterIndeterminateShuffle(0) - masterTracker.registerIndeterminateShuffle(0, 1) - masterTracker.incrementEpoch() - slaveTracker.updateEpoch(masterTracker.getEpoch) - - // Update worker side indeterminateAttemptIds - slaveTracker.getMapSizesByExecutorId(0, 0, 1, needFetchIndeterminateAttemptId = true) - assert(slaveTracker.indeterminateAttemptIds(0).get == 1) - - masterTracker.stop() - slaveTracker.stop() - rpcEnv.shutdown() - slaveRpcEnv.shutdown() - } - } diff --git a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala index b187d7138218..32678ab25a84 100644 --- a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala +++ b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala @@ -420,18 +420,18 @@ abstract class ShuffleSuite extends SparkFunSuite with Matchers with LocalSparkC // Cannot find one of the local shuffle blocks. val dataFile = sc.env.blockManager.diskBlockManager.getFile( - new ShuffleDataBlockId(0, 0, 0, Some(0))) + new ShuffleDataBlockId(0, 0, 0, 0)) val indexFile = sc.env.blockManager.diskBlockManager.getFile( - new ShuffleIndexBlockId(0, 0, 0, Some(0))) + new ShuffleIndexBlockId(0, 0, 0, 0)) assert(!dataFile.exists() && !indexFile.exists()) rdd.count() // Can find one of the local shuffle blocks. val dataExistsFile = sc.env.blockManager.diskBlockManager.getFile( - new ShuffleDataBlockId(0, 0, 0, Some(0))) + new ShuffleDataBlockId(0, 0, 0, 0)) val indexExistsFile = sc.env.blockManager.diskBlockManager.getFile( - new ShuffleIndexBlockId(0, 0, 0, Some(0))) + new ShuffleIndexBlockId(0, 0, 0, 0)) assert(dataExistsFile.exists() && indexExistsFile.exists()) } } 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 c1e7fb9a1db1..58aa0f10feeb 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/MapStatusSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/MapStatusSuite.scala @@ -87,7 +87,7 @@ 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 status = MapStatus(loc, sizes, 1) val status1 = compressAndDecompressMapStatus(status) assert(status1.isInstanceOf[HighlyCompressedMapStatus]) assert(status1.location == loc) @@ -97,6 +97,7 @@ class MapStatusSuite extends SparkFunSuite { assert(estimate === avg) } } + assert(status1.stageAttemptId == 1) } test("SPARK-22540: ensure HighlyCompressedMapStatus calculates correct avgSize") { @@ -109,7 +110,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, 1) val status1 = compressAndDecompressMapStatus(status) assert(status1.isInstanceOf[HighlyCompressedMapStatus]) assert(status1.location == loc) @@ -119,6 +120,7 @@ class MapStatusSuite extends SparkFunSuite { assert(estimate === avg) } } + assert(status1.stageAttemptId == 1) } def compressAndDecompressMapStatus(status: MapStatus): MapStatus = { 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 2442670b6d3f..e012597cb184 100644 --- a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala @@ -351,7 +351,8 @@ class KryoSerializerSuite extends SparkFunSuite with SharedSparkContext { val denseBlockSizes = new Array[Long](5000) val sparseBlockSizes = Array[Long](0L, 1L, 0L, 2L) Seq(denseBlockSizes, sparseBlockSizes).foreach { blockSizes => - ser.serialize(HighlyCompressedMapStatus(BlockManagerId("exec-1", "host", 1234), blockSizes)) + ser.serialize(HighlyCompressedMapStatus( + BlockManagerId("exec-1", "host", 1234), blockSizes, 0)) } } 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 58b100ffd713..f6bc4d8674cf 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 @@ -67,6 +67,7 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte when(dependency.partitioner).thenReturn(new HashPartitioner(7)) when(dependency.serializer).thenReturn(new JavaSerializer(conf)) when(taskContext.taskMetrics()).thenReturn(taskMetrics) +<<<<<<< HEAD when(taskContext.getIndeterminateAttemptId).thenReturn(None) when(blockResolver.getDataFile(0, 0, None)).thenReturn(outputFile) doAnswer { (invocationOnMock: InvocationOnMock) => @@ -74,11 +75,23 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte if (tmp != null) { outputFile.delete tmp.renameTo(outputFile) +======= + when(taskContext.stageAttemptNumber()).thenReturn(0) + when(blockResolver.getDataFile(0, 0, 0)).thenReturn(outputFile) + doAnswer(new Answer[Void] { + def answer(invocationOnMock: InvocationOnMock): Void = { + val tmp: File = invocationOnMock.getArguments()(3).asInstanceOf[File] + if (tmp != null) { + outputFile.delete + tmp.renameTo(outputFile) + } + null +>>>>>>> extend all shuffle block id, not only the indeterminate ones. } null }.when(blockResolver) .writeIndexFileAndCommit( - anyInt, anyInt, any(classOf[Array[Long]]), any(classOf[File]), any(classOf[Option[Int]])) + anyInt, anyInt, any(classOf[Array[Long]]), any(classOf[File]), any(classOf[Int])) when(blockManager.diskBlockManager).thenReturn(diskBlockManager) when(blockManager.getDiskWriter( any[BlockId], 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 363cd30fd53f..79941d6878cb 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 @@ -59,7 +59,7 @@ class IndexShuffleBlockResolverSuite extends SparkFunSuite with BeforeAndAfterEa } private def testWithIndexShuffleBlockResolver( - shuffleId: Int, mapId: Int, idxName: String, attemptId: Option[Int]): Unit = { + shuffleId: Int, mapId: Int, idxName: String, attemptId: Int): Unit = { val resolver = new IndexShuffleBlockResolver(conf, blockManager) val lengths = Array[Long](10, 0, 20) val dataTmp = File.createTempFile("shuffle", null, tempDir) @@ -157,14 +157,14 @@ 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" - testWithIndexShuffleBlockResolver(shuffleId, mapId, idxName, None) + val idxName = s"shuffle_${shuffleId}_${mapId}_0_0.index" + testWithIndexShuffleBlockResolver(shuffleId, mapId, idxName, 0) } test("commit shuffle files with attempt id multiple times") { val shuffleId = 1 val mapId = 2 - val attemptId = Some(1) + val attemptId = 1 val idxName = s"shuffle_${shuffleId}_${mapId}_0_1.index" testWithIndexShuffleBlockResolver(shuffleId, mapId, idxName, attemptId) } 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 7bf510795a05..e78d0045a635 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockIdSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockIdSuite.scala @@ -55,7 +55,7 @@ class BlockIdSuite extends SparkFunSuite { val id = ShuffleBlockId(1, 2, 3) assertSame(id, ShuffleBlockId(1, 2, 3)) assertDifferent(id, ShuffleBlockId(3, 2, 3)) - assert(id.name === "shuffle_1_2_3") + assert(id.name === "shuffle_1_2_3_0") assert(id.asRDDId === None) assert(id.shuffleId === 1) assert(id.mapId === 2) @@ -68,7 +68,7 @@ class BlockIdSuite extends SparkFunSuite { val id = ShuffleDataBlockId(4, 5, 6) assertSame(id, ShuffleDataBlockId(4, 5, 6)) assertDifferent(id, ShuffleDataBlockId(6, 5, 6)) - assert(id.name === "shuffle_4_5_6.data") + assert(id.name === "shuffle_4_5_6_0.data") assert(id.asRDDId === None) assert(id.shuffleId === 4) assert(id.mapId === 5) @@ -81,7 +81,7 @@ class BlockIdSuite extends SparkFunSuite { val id = ShuffleIndexBlockId(7, 8, 9) assertSame(id, ShuffleIndexBlockId(7, 8, 9)) assertDifferent(id, ShuffleIndexBlockId(9, 8, 9)) - assert(id.name === "shuffle_7_8_9.index") + assert(id.name === "shuffle_7_8_9_0.index") assert(id.asRDDId === None) assert(id.shuffleId === 7) assert(id.mapId === 8) @@ -91,43 +91,43 @@ class BlockIdSuite extends SparkFunSuite { } test("extend shuffle") { - val id = ShuffleBlockId(1, 2, 3, Some(4)) - assertSame(id, ShuffleBlockId(1, 2, 3, Some(4))) - assertDifferent(id, ShuffleBlockId(3, 2, 3, Some(5))) + val id = ShuffleBlockId(1, 2, 3, 4) + assertSame(id, ShuffleBlockId(1, 2, 3, 4)) + assertDifferent(id, ShuffleBlockId(3, 2, 3, 5)) assert(id.name === "shuffle_1_2_3_4") assert(id.asRDDId === None) assert(id.shuffleId === 1) assert(id.mapId === 2) assert(id.reduceId === 3) - assert(id.indeterminateAttemptId.get === 4) + assert(id.stageAttemptId === 4) assert(id.isShuffle) assertSame(id, BlockId(id.toString)) } test("extend shuffle data") { - val id = ShuffleDataBlockId(4, 5, 6, Some(7)) - assertSame(id, ShuffleDataBlockId(4, 5, 6, Some(7))) - assertDifferent(id, ShuffleDataBlockId(6, 5, 6, Some(8))) + val id = ShuffleDataBlockId(4, 5, 6, 7) + assertSame(id, ShuffleDataBlockId(4, 5, 6, 7)) + assertDifferent(id, ShuffleDataBlockId(6, 5, 6, 8)) assert(id.name === "shuffle_4_5_6_7.data") assert(id.asRDDId === None) assert(id.shuffleId === 4) assert(id.mapId === 5) assert(id.reduceId === 6) - assert(id.indeterminateAttemptId.get === 7) + assert(id.stageAttemptId === 7) assert(!id.isShuffle) assertSame(id, BlockId(id.toString)) } test("extend shuffle index") { - val id = ShuffleIndexBlockId(7, 8, 9, Some(10)) - assertSame(id, ShuffleIndexBlockId(7, 8, 9, Some(10))) - assertDifferent(id, ShuffleIndexBlockId(9, 8, 9, Some(11))) + val id = ShuffleIndexBlockId(7, 8, 9, 10) + assertSame(id, ShuffleIndexBlockId(7, 8, 9, 10)) + assertDifferent(id, ShuffleIndexBlockId(9, 8, 9, 11)) assert(id.name === "shuffle_7_8_9_10.index") assert(id.asRDDId === None) assert(id.shuffleId === 7) assert(id.mapId === 8) assert(id.reduceId === 9) - assert(id.indeterminateAttemptId.get === 10) + assert(id.stageAttemptId === 10) assert(!id.isShuffle) assertSame(id, BlockId(id.toString)) } diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 9f3aaf2f34b0..776f3399240f 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -271,8 +271,6 @@ object MimaExcludes { ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.status.api.v1.ApplicationEnvironmentInfo.this"), // [SPARK-25341][CORE] Support rolling back a shuffle map stage and re-generate the shuffle files - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.TaskContext.getIndeterminateAttemptId"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.TaskContext.markAsIndeterminate"), ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.storage.ShuffleIndexBlockId.copy"), ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.storage.ShuffleIndexBlockId.this"), ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.storage.ShuffleDataBlockId.copy"), From 3b61eba157c2c02cd062a56b9f06e390b59864da Mon Sep 17 00:00:00 2001 From: Yuanjian Li Date: Thu, 28 Mar 2019 22:49:48 +0800 Subject: [PATCH 10/16] rebase and fix build --- .../apache/spark/deploy/ExternalShuffleServiceDbSuite.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/deploy/ExternalShuffleServiceDbSuite.scala b/core/src/test/scala/org/apache/spark/deploy/ExternalShuffleServiceDbSuite.scala index e33c3f8f9550..bd77366d8d9b 100644 --- a/core/src/test/scala/org/apache/spark/deploy/ExternalShuffleServiceDbSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/ExternalShuffleServiceDbSuite.scala @@ -100,7 +100,7 @@ class ExternalShuffleServiceDbSuite extends SparkFunSuite { blockHandler = externalShuffleService.getBlockHandler blockResolver = blockHandler.getBlockResolver - val block0Stream = blockResolver.getBlockData("app0", "exec0", 0, 0, 0).createInputStream + val block0Stream = blockResolver.getBlockData("app0", "exec0", 0, 0, 0, 0).createInputStream val block0 = CharStreams.toString(new InputStreamReader(block0Stream, StandardCharsets.UTF_8)) block0Stream.close() assert(sortBlock0 == block0) @@ -127,7 +127,7 @@ class ExternalShuffleServiceDbSuite extends SparkFunSuite { blockResolver = blockHandler.getBlockResolver val error = intercept[RuntimeException] { - blockResolver.getBlockData("app0", "exec0", 0, 0, 0).createInputStream + blockResolver.getBlockData("app0", "exec0", 0, 0, 0, 0).createInputStream }.getMessage assert(error.contains("not registered")) From fbe245d7f425f8f3e869cbfb90609477483bc680 Mon Sep 17 00:00:00 2001 From: Yuanjian Li Date: Wed, 3 Apr 2019 23:54:22 +0800 Subject: [PATCH 11/16] reimplement by using the local properties of Task --- .../shuffle/ExternalShuffleBlockHandler.java | 10 ++-- .../shuffle/ExternalShuffleBlockResolver.java | 30 +++++++--- .../network/sasl/SaslIntegrationSuite.java | 2 +- .../ExternalShuffleBlockHandlerSuite.java | 10 ++-- .../ExternalShuffleBlockResolverSuite.java | 36 ++++++++++-- .../shuffle/ExternalShuffleCleanupSuite.java | 2 +- .../ExternalShuffleIntegrationSuite.java | 55 +++++++++++++----- .../shuffle/NonShuffleFilesCleanupSuite.java | 2 +- .../shuffle/TestShuffleDataContext.java | 6 +- .../sort/BypassMergeSortShuffleWriter.java | 10 ++-- .../shuffle/sort/UnsafeShuffleWriter.java | 7 +-- .../org/apache/spark/MapOutputTracker.scala | 36 ++++++++---- .../scala/org/apache/spark/SparkContext.scala | 3 + .../scala/org/apache/spark/TaskContext.scala | 11 ++++ .../apache/spark/scheduler/DAGScheduler.scala | 35 ++++++++--- .../apache/spark/scheduler/MapStatus.scala | 44 ++++---------- .../org/apache/spark/scheduler/TaskSet.scala | 3 +- .../shuffle/BlockStoreShuffleReader.scala | 3 +- .../shuffle/IndexShuffleBlockResolver.scala | 24 ++++---- .../shuffle/sort/SortShuffleManager.scala | 20 +++---- .../shuffle/sort/SortShuffleWriter.scala | 10 ++-- .../org/apache/spark/storage/BlockId.scala | 58 +++++++++++++------ .../sort/UnsafeShuffleWriterSuite.java | 4 +- .../apache/spark/MapOutputTrackerSuite.scala | 2 +- .../scala/org/apache/spark/ShuffleSuite.scala | 22 ------- .../ExternalShuffleServiceDbSuite.scala | 6 +- .../spark/scheduler/DAGSchedulerSuite.scala | 15 ++++- .../spark/scheduler/MapStatusSuite.scala | 6 +- .../serializer/KryoSerializerSuite.scala | 2 +- .../BlockStoreShuffleReaderSuite.scala | 3 +- .../BypassMergeSortShuffleWriterSuite.scala | 17 +----- .../sort/IndexShuffleBlockResolverSuite.scala | 8 +-- .../apache/spark/storage/BlockIdSuite.scala | 30 +++++----- 33 files changed, 309 insertions(+), 223 deletions(-) diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java index 4a0dc09e8b2a..648aa94e928a 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java @@ -218,7 +218,7 @@ private class ManagedBufferIterator implements Iterator { private final int shuffleId; // An array containing mapId and reduceId pairs. private final int[] mapIdAndReduceIds; - private final int stageAttemptId; + private final int indeterminateAttemptId; ManagedBufferIterator(String appId, String execId, String[] blockIds) { this.appId = appId; @@ -227,6 +227,8 @@ private class ManagedBufferIterator implements Iterator { checkBlockId(blockId0Parts, blockIds, 0); this.shuffleId = Integer.parseInt(blockId0Parts[1]); mapIdAndReduceIds = new int[2 * blockIds.length]; + this.indeterminateAttemptId = + (blockId0Parts.length == 5) ? Integer.parseInt(blockId0Parts[4]) : -1; for (int i = 0; i < blockIds.length; i++) { String[] blockIdParts = blockIds[i].split("_"); checkBlockId(blockIdParts, blockIds, i); @@ -237,11 +239,11 @@ private class ManagedBufferIterator implements Iterator { mapIdAndReduceIds[2 * i] = Integer.parseInt(blockIdParts[2]); mapIdAndReduceIds[2 * i + 1] = Integer.parseInt(blockIdParts[3]); } - this.stageAttemptId = Integer.parseInt(blockId0Parts[4]); } private void checkBlockId(String[] blockIdParts, String[] wholeBlockId, int index) { - if (blockIdParts.length != 5 || !blockIdParts[0].equals("shuffle")) { + if ((blockIdParts.length != 4 && blockIdParts.length != 5) + || !blockIdParts[0].equals("shuffle")) { throw new IllegalArgumentException( "Unexpected shuffle block id format: " + wholeBlockId[index]); } @@ -255,7 +257,7 @@ public boolean hasNext() { @Override public ManagedBuffer next() { final ManagedBuffer block = blockManager.getBlockData(appId, execId, shuffleId, - mapIdAndReduceIds[index], mapIdAndReduceIds[index + 1], stageAttemptId); + mapIdAndReduceIds[index], mapIdAndReduceIds[index + 1], indeterminateAttemptId); index += 2; metrics.blockTransferRateBytes.mark(block != null ? block.size() : 0); return block; 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 7823625f869c..74bcdf4e5082 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 @@ -161,6 +161,18 @@ public void registerExecutor( executors.put(fullId, executorInfo); } + /** + * Overload getBlockData with setting stageAttemptId to an invalid value of -1. + */ + public ManagedBuffer getBlockData( + String appId, + String execId, + int shuffleId, + int mapId, + int reduceId) { + return getBlockData(appId, execId, shuffleId, mapId, reduceId, -1); + } + /** * Obtains a FileSegmentManagedBuffer from (shuffleId, mapId, reduceId). We make assumptions * about how the hash and sort based shuffles store their data. @@ -277,15 +289,19 @@ public boolean accept(File dir, String name) { } /** - * Sort-based shuffle data uses an index called "shuffle_ShuffleId_MapId_0_StageAttemptId.index" - * into a data file called "shuffle_ShuffleId_MapId_0_StageAttemptId.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. + * While the shuffle data and index file generated from the indeterminate stage, + * the ShuffleDataBlockId and ShuffleIndexBlockId will be extended by the stage attempt id. */ private ManagedBuffer getSortBasedShuffleBlockData( - ExecutorShuffleInfo executor, int shuffleId, - int mapId, int reduceId, int stageAttemptId) { - String baseFileName = "shuffle_" + shuffleId + "_" + mapId + "_0" + "_" + stageAttemptId; + ExecutorShuffleInfo executor, int shuffleId, + int mapId, int reduceId, int stageAttemptId) { + String baseFileName = "shuffle_" + shuffleId + "_" + mapId + "_0"; + if (stageAttemptId != -1) { + baseFileName = baseFileName + "_" + stageAttemptId; + } File indexFile = getFile(executor.localDirs, executor.subDirsPerLocalDir, baseFileName + ".index"); diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/sasl/SaslIntegrationSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/sasl/SaslIntegrationSuite.java index 913f6a7c451f..57c1c5e7722c 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/sasl/SaslIntegrationSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/sasl/SaslIntegrationSuite.java @@ -201,7 +201,7 @@ public void onBlockFetchFailure(String blockId, Throwable t) { } }; - String[] blockIds = { "shuffle_0_1_2_0", "shuffle_0_3_4_0" }; + String[] blockIds = { "shuffle_0_1_2", "shuffle_0_3_4" }; OneForOneBlockFetcher fetcher = new OneForOneBlockFetcher(client1, "app-2", "0", blockIds, listener, conf); fetcher.start(); diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandlerSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandlerSuite.java index 2c426bf584e3..63344aed63f8 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandlerSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandlerSuite.java @@ -85,10 +85,10 @@ public void testOpenShuffleBlocks() { ManagedBuffer block0Marker = new NioManagedBuffer(ByteBuffer.wrap(new byte[3])); ManagedBuffer block1Marker = new NioManagedBuffer(ByteBuffer.wrap(new byte[7])); - when(blockResolver.getBlockData("app0", "exec1", 0, 0, 0, 0)).thenReturn(block0Marker); - when(blockResolver.getBlockData("app0", "exec1", 0, 0, 1, 0)).thenReturn(block1Marker); + when(blockResolver.getBlockData("app0", "exec1", 0, 0, 0, -1)).thenReturn(block0Marker); + when(blockResolver.getBlockData("app0", "exec1", 0, 0, 1, -1)).thenReturn(block1Marker); ByteBuffer openBlocks = new OpenBlocks("app0", "exec1", - new String[] { "shuffle_0_0_0_0", "shuffle_0_0_1_0" }) + new String[] { "shuffle_0_0_0", "shuffle_0_0_1" }) .toByteBuffer(); handler.receive(client, openBlocks, callback); @@ -109,8 +109,8 @@ public void testOpenShuffleBlocks() { assertEquals(block0Marker, buffers.next()); assertEquals(block1Marker, buffers.next()); assertFalse(buffers.hasNext()); - verify(blockResolver, times(1)).getBlockData("app0", "exec1", 0, 0, 0, 0); - verify(blockResolver, times(1)).getBlockData("app0", "exec1", 0, 0, 1, 0); + verify(blockResolver, times(1)).getBlockData("app0", "exec1", 0, 0, 0, -1); + verify(blockResolver, times(1)).getBlockData("app0", "exec1", 0, 0, 1, -1); // Verify open block request latency metrics Timer openBlockRequestLatencyMillis = (Timer) ((ExternalShuffleBlockHandler) handler) diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolverSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolverSuite.java index b18c353ee02c..3b7cee2b0145 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolverSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolverSuite.java @@ -53,7 +53,10 @@ public static void beforeAll() throws IOException { // Write some sort data. dataContext.insertSortShuffleData(0, 0, new byte[][] { sortBlock0.getBytes(StandardCharsets.UTF_8), - sortBlock1.getBytes(StandardCharsets.UTF_8)}); + sortBlock1.getBytes(StandardCharsets.UTF_8)}, false); + dataContext.insertSortShuffleData(0, 0, new byte[][] { + sortBlock0.getBytes(StandardCharsets.UTF_8), + sortBlock1.getBytes(StandardCharsets.UTF_8)}, true); } @AfterClass @@ -66,7 +69,7 @@ public void testBadRequests() throws IOException { ExternalShuffleBlockResolver resolver = new ExternalShuffleBlockResolver(conf, null); // Unregistered executor try { - resolver.getBlockData("app0", "exec1", 1, 1, 0, 0); + resolver.getBlockData("app0", "exec1", 1, 1, 0); fail("Should have failed"); } catch (RuntimeException e) { assertTrue("Bad error message: " + e, e.getMessage().contains("not registered")); @@ -75,7 +78,7 @@ public void testBadRequests() throws IOException { // Invalid shuffle manager try { resolver.registerExecutor("app0", "exec2", dataContext.createExecutorInfo("foobar")); - resolver.getBlockData("app0", "exec2", 1, 1, 0, 0); + resolver.getBlockData("app0", "exec2", 1, 1, 0); fail("Should have failed"); } catch (UnsupportedOperationException e) { // pass @@ -85,7 +88,7 @@ public void testBadRequests() throws IOException { resolver.registerExecutor("app0", "exec3", dataContext.createExecutorInfo(SORT_MANAGER)); try { - resolver.getBlockData("app0", "exec3", 1, 1, 0, 0); + resolver.getBlockData("app0", "exec3", 1, 1, 0); fail("Should have failed"); } catch (Exception e) { // pass @@ -99,20 +102,41 @@ public void testSortShuffleBlocks() throws IOException { dataContext.createExecutorInfo(SORT_MANAGER)); try (InputStream block0Stream = resolver.getBlockData( - "app0", "exec0", 0, 0, 0, 0).createInputStream()) { + "app0", "exec0", 0, 0, 0).createInputStream()) { String block0 = CharStreams.toString(new InputStreamReader(block0Stream, StandardCharsets.UTF_8)); assertEquals(sortBlock0, block0); } try (InputStream block1Stream = resolver.getBlockData( - "app0", "exec0", 0, 0, 1, 0).createInputStream()) { + "app0", "exec0", 0, 0, 1).createInputStream()) { String block1 = CharStreams.toString(new InputStreamReader(block1Stream, StandardCharsets.UTF_8)); assertEquals(sortBlock1, block1); } } + @Test + public void testExtendedSortShuffleBlocks() throws IOException { + ExternalShuffleBlockResolver resolver = new ExternalShuffleBlockResolver(conf, null); + resolver.registerExecutor("app0", "exec0", + dataContext.createExecutorInfo(SORT_MANAGER)); + + try (InputStream block0Stream = resolver.getBlockData( + "app0", "exec0", 0, 0, 0, 0).createInputStream()) { + String block0 = + CharStreams.toString(new InputStreamReader(block0Stream, StandardCharsets.UTF_8)); + assertEquals(sortBlock0, block0); + } + + try (InputStream block1Stream = resolver.getBlockData( + "app0", "exec0", 0, 0, 1, 0).createInputStream()) { + String block1 = + CharStreams.toString(new InputStreamReader(block1Stream, StandardCharsets.UTF_8)); + assertEquals(sortBlock1, block1); + } + } + @Test public void jsonSerializationOfExecutorRegistration() throws IOException { ObjectMapper mapper = new ObjectMapper(); diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleCleanupSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleCleanupSuite.java index 47c087088a8a..259cc2e04b7d 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleCleanupSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleCleanupSuite.java @@ -142,7 +142,7 @@ private static TestShuffleDataContext createSomeData() throws IOException { dataContext.create(); dataContext.insertSortShuffleData(rand.nextInt(1000), rand.nextInt(1000), new byte[][] { "ABC".getBytes(StandardCharsets.UTF_8), - "DEF".getBytes(StandardCharsets.UTF_8)}); + "DEF".getBytes(StandardCharsets.UTF_8)}, false); return dataContext; } } diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java index 4e490422c366..516e24afdaef 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java @@ -84,7 +84,8 @@ public static void beforeAll() throws IOException { dataContext0 = new TestShuffleDataContext(2, 5); dataContext0.create(); - dataContext0.insertSortShuffleData(0, 0, exec0Blocks); + dataContext0.insertSortShuffleData(0, 0, exec0Blocks, false); + dataContext0.insertSortShuffleData(0, 0, exec0Blocks, true); conf = new TransportConf("shuffle", MapConfigProvider.EMPTY); handler = new ExternalShuffleBlockHandler(conf, null); @@ -171,6 +172,28 @@ public void onBlockFetchFailure(String blockId, Throwable exception) { @Test public void testFetchOneSort() throws Exception { + registerExecutor("exec-0", dataContext0.createExecutorInfo(SORT_MANAGER)); + FetchResult exec0Fetch = fetchBlocks("exec-0", new String[] { "shuffle_0_0_0" }); + assertEquals(Sets.newHashSet("shuffle_0_0_0"), exec0Fetch.successBlocks); + assertTrue(exec0Fetch.failedBlocks.isEmpty()); + assertBufferListsEqual(exec0Fetch.buffers, Arrays.asList(exec0Blocks[0])); + exec0Fetch.releaseBuffers(); + } + + @Test + public void testFetchThreeSort() throws Exception { + registerExecutor("exec-0", dataContext0.createExecutorInfo(SORT_MANAGER)); + FetchResult exec0Fetch = fetchBlocks("exec-0", + new String[] { "shuffle_0_0_0", "shuffle_0_0_1", "shuffle_0_0_2" }); + assertEquals(Sets.newHashSet("shuffle_0_0_0", "shuffle_0_0_1", "shuffle_0_0_2"), + exec0Fetch.successBlocks); + assertTrue(exec0Fetch.failedBlocks.isEmpty()); + assertBufferListsEqual(exec0Fetch.buffers, Arrays.asList(exec0Blocks)); + exec0Fetch.releaseBuffers(); + } + + @Test + public void testFetchOneExtendedSort() throws Exception { registerExecutor("exec-0", dataContext0.createExecutorInfo(SORT_MANAGER)); FetchResult exec0Fetch = fetchBlocks("exec-0", new String[] { "shuffle_0_0_0_0" }); assertEquals(Sets.newHashSet("shuffle_0_0_0_0"), exec0Fetch.successBlocks); @@ -180,12 +203,12 @@ public void testFetchOneSort() throws Exception { } @Test - public void testFetchThreeSort() throws Exception { + public void testFetchThreeExtendedSort() throws Exception { registerExecutor("exec-0", dataContext0.createExecutorInfo(SORT_MANAGER)); FetchResult exec0Fetch = fetchBlocks("exec-0", - new String[] { "shuffle_0_0_0_0", "shuffle_0_0_1_0", "shuffle_0_0_2_0" }); + new String[] { "shuffle_0_0_0_0", "shuffle_0_0_1_0", "shuffle_0_0_2_0" }); assertEquals(Sets.newHashSet("shuffle_0_0_0_0", "shuffle_0_0_1_0", "shuffle_0_0_2_0"), - exec0Fetch.successBlocks); + exec0Fetch.successBlocks); assertTrue(exec0Fetch.failedBlocks.isEmpty()); assertBufferListsEqual(exec0Fetch.buffers, Arrays.asList(exec0Blocks)); exec0Fetch.releaseBuffers(); @@ -199,36 +222,36 @@ public void testRegisterInvalidExecutor() throws Exception { @Test public void testFetchWrongBlockId() throws Exception { registerExecutor("exec-1", dataContext0.createExecutorInfo(SORT_MANAGER)); - FetchResult execFetch = fetchBlocks("exec-1", new String[] { "rdd_1_0_0_0" }); + FetchResult execFetch = fetchBlocks("exec-1", new String[] { "rdd_1_0_0" }); assertTrue(execFetch.successBlocks.isEmpty()); - assertEquals(Sets.newHashSet("rdd_1_0_0_0"), execFetch.failedBlocks); + assertEquals(Sets.newHashSet("rdd_1_0_0"), execFetch.failedBlocks); } @Test public void testFetchNonexistent() throws Exception { registerExecutor("exec-0", dataContext0.createExecutorInfo(SORT_MANAGER)); FetchResult execFetch = fetchBlocks("exec-0", - new String[] { "shuffle_2_0_0_0" }); + new String[] { "shuffle_2_0_0" }); assertTrue(execFetch.successBlocks.isEmpty()); - assertEquals(Sets.newHashSet("shuffle_2_0_0_0"), execFetch.failedBlocks); + assertEquals(Sets.newHashSet("shuffle_2_0_0"), execFetch.failedBlocks); } @Test public void testFetchWrongExecutor() throws Exception { registerExecutor("exec-0", dataContext0.createExecutorInfo(SORT_MANAGER)); - FetchResult execFetch0 = fetchBlocks("exec-0", new String[] { "shuffle_0_0_0_0" /* right */}); - FetchResult execFetch1 = fetchBlocks("exec-0", new String[] { "shuffle_1_0_0_0" /* wrong */ }); - assertEquals(Sets.newHashSet("shuffle_0_0_0_0"), execFetch0.successBlocks); - assertEquals(Sets.newHashSet("shuffle_1_0_0_0"), execFetch1.failedBlocks); + FetchResult execFetch0 = fetchBlocks("exec-0", new String[] { "shuffle_0_0_0" /* right */}); + FetchResult execFetch1 = fetchBlocks("exec-0", new String[] { "shuffle_1_0_0" /* wrong */ }); + assertEquals(Sets.newHashSet("shuffle_0_0_0"), execFetch0.successBlocks); + assertEquals(Sets.newHashSet("shuffle_1_0_0"), execFetch1.failedBlocks); } @Test public void testFetchUnregisteredExecutor() throws Exception { registerExecutor("exec-0", dataContext0.createExecutorInfo(SORT_MANAGER)); FetchResult execFetch = fetchBlocks("exec-2", - new String[] { "shuffle_0_0_0_0", "shuffle_1_0_0_0" }); + new String[] { "shuffle_0_0_0", "shuffle_1_0_0" }); assertTrue(execFetch.successBlocks.isEmpty()); - assertEquals(Sets.newHashSet("shuffle_0_0_0_0", "shuffle_1_0_0_0"), execFetch.failedBlocks); + assertEquals(Sets.newHashSet("shuffle_0_0_0", "shuffle_1_0_0"), execFetch.failedBlocks); } @Test @@ -237,9 +260,9 @@ public void testFetchNoServer() throws Exception { new MapConfigProvider(ImmutableMap.of("spark.shuffle.io.maxRetries", "0"))); registerExecutor("exec-0", dataContext0.createExecutorInfo(SORT_MANAGER)); FetchResult execFetch = fetchBlocks("exec-0", - new String[]{"shuffle_1_0_0_0", "shuffle_1_0_1_0"}, clientConf, 1 /* port */); + new String[]{"shuffle_1_0_0", "shuffle_1_0_1"}, clientConf, 1 /* port */); assertTrue(execFetch.successBlocks.isEmpty()); - assertEquals(Sets.newHashSet("shuffle_1_0_0_0", "shuffle_1_0_1_0"), execFetch.failedBlocks); + assertEquals(Sets.newHashSet("shuffle_1_0_0", "shuffle_1_0_1"), execFetch.failedBlocks); } private static void registerExecutor(String executorId, ExecutorShuffleInfo executorInfo) diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/NonShuffleFilesCleanupSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/NonShuffleFilesCleanupSuite.java index d22f3ace4103..80b27010d903 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/NonShuffleFilesCleanupSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/NonShuffleFilesCleanupSuite.java @@ -211,7 +211,7 @@ private static void createShuffleFiles(TestShuffleDataContext dataContext) throw Random rand = new Random(123); dataContext.insertSortShuffleData(rand.nextInt(1000), rand.nextInt(1000), new byte[][] { "ABC".getBytes(StandardCharsets.UTF_8), - "DEF".getBytes(StandardCharsets.UTF_8)}); + "DEF".getBytes(StandardCharsets.UTF_8)}, false); } private static void createNonShuffleFiles(TestShuffleDataContext dataContext) throws IOException { diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/TestShuffleDataContext.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/TestShuffleDataContext.java index 84643683fa61..f0904fbc42a1 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/TestShuffleDataContext.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/TestShuffleDataContext.java @@ -68,8 +68,10 @@ public void cleanup() { } /** Creates reducer blocks in a sort-based data format within our local dirs. */ - public void insertSortShuffleData(int shuffleId, int mapId, byte[][] blocks) throws IOException { - String blockId = "shuffle_" + shuffleId + "_" + mapId + "_0" + "_0"; + public void insertSortShuffleData( + int shuffleId, int mapId, byte[][] blocks, boolean extendedBlockId) throws IOException { + String blockId = "shuffle_" + shuffleId + "_" + mapId + "_0"; + if (extendedBlockId) blockId += "_0"; OutputStream dataStream = null; DataOutputStream indexStream = null; 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 fd962cf566a2..ba0bdfe9a7b9 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 @@ -84,7 +84,7 @@ final class BypassMergeSortShuffleWriter extends ShuffleWriter { private final int mapId; private final Serializer serializer; private final IndexShuffleBlockResolver shuffleBlockResolver; - private final int stageAttemptId; + private final Option stageAttemptId; /** Array of file writers, one for each partition */ private DiskBlockObjectWriter[] partitionWriters; @@ -119,7 +119,7 @@ final class BypassMergeSortShuffleWriter extends ShuffleWriter { this.writeMetrics = writeMetrics; this.serializer = dep.serializer(); this.shuffleBlockResolver = shuffleBlockResolver; - this.stageAttemptId = taskContext.stageAttemptNumber(); + this.stageAttemptId = taskContext.indeterminateStageAttemptId(dep.shuffleId()); } @Override @@ -129,8 +129,7 @@ public void write(Iterator> records) throws IOException { partitionLengths = new long[numPartitions]; shuffleBlockResolver.writeIndexFileAndCommit( shuffleId, mapId, partitionLengths, null, stageAttemptId); - mapStatus = MapStatus$.MODULE$.apply( - blockManager.shuffleServerId(), partitionLengths, stageAttemptId); + mapStatus = MapStatus$.MODULE$.apply(blockManager.shuffleServerId(), partitionLengths); return; } final SerializerInstance serInstance = serializer.newInstance(); @@ -173,8 +172,7 @@ public void write(Iterator> records) throws IOException { logger.error("Error while deleting temp file {}", tmp.getAbsolutePath()); } } - mapStatus = MapStatus$.MODULE$.apply( - blockManager.shuffleServerId(), partitionLengths, stageAttemptId); + mapStatus = MapStatus$.MODULE$.apply(blockManager.shuffleServerId(), partitionLengths); } @VisibleForTesting 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 464978f7d48b..a2c764343d82 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 @@ -82,7 +82,7 @@ public class UnsafeShuffleWriter extends ShuffleWriter { private final int initialSortBufferSize; private final int inputBufferSizeInBytes; private final int outputBufferSizeInBytes; - private final int stageAttemptId; + private final Option stageAttemptId; @Nullable private MapStatus mapStatus; @Nullable private ShuffleExternalSorter sorter; @@ -151,7 +151,7 @@ public UnsafeShuffleWriter( this.outputBufferSizeInBytes = (int) (long) sparkConf.get(package$.MODULE$.SHUFFLE_UNSAFE_FILE_OUTPUT_BUFFER_SIZE()) * 1024; open(); - this.stageAttemptId = taskContext.stageAttemptNumber(); + this.stageAttemptId = taskContext.indeterminateStageAttemptId(dep.shuffleId()); } private void updatePeakMemoryUsed() { @@ -253,8 +253,7 @@ void closeAndWriteOutput() throws IOException { logger.error("Error while deleting temp file {}", tmp.getAbsolutePath()); } } - mapStatus = MapStatus$.MODULE$.apply( - blockManager.shuffleServerId(), partitionLengths, stageAttemptId); + mapStatus = MapStatus$.MODULE$.apply(blockManager.shuffleServerId(), partitionLengths); } @VisibleForTesting diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index e2505a3abe6c..c439574955b7 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -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) 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)])] + def getMapSizesByExecutorId( + shuffleId: Int, + startPartition: Int, + endPartition: Int, + stageAttemptId: Option[Int] = None) + : Iterator[(BlockManagerId, Seq[(BlockId, Long)])] /** * 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)])] = { + def getMapSizesByExecutorId( + shuffleId: Int, + startPartition: Int, + endPartition: Int, + stageAttemptId: Option[Int] = None) + : Iterator[(BlockManagerId, Seq[(BlockId, Long)])] = { 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, stageAttemptId) } case None => Iterator.empty @@ -682,12 +691,17 @@ private[spark] class MapOutputTrackerWorker(conf: SparkConf) extends MapOutputTr private val fetching = new HashSet[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)])] = { + override def getMapSizesByExecutorId( + shuffleId: Int, + startPartition: Int, + endPartition: Int, + stageAttemptId: Option[Int] = None) + : Iterator[(BlockManagerId, Seq[(BlockId, Long)])] = { 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, stageAttemptId) } catch { case e: MetadataFetchFailedException => // We experienced a fetch failure so our mapStatuses cache is outdated; clear it: @@ -863,6 +877,7 @@ 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 stageAttemptId The stage attempt id for retried indeterminate stage. * @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. @@ -871,7 +886,8 @@ private[spark] object MapOutputTracker extends Logging { shuffleId: Int, startPartition: Int, endPartition: Int, - statuses: Array[MapStatus]): Iterator[(BlockManagerId, Seq[(BlockId, Long)])] = { + statuses: Array[MapStatus], + stageAttemptId: Option[Int]): Iterator[(BlockManagerId, Seq[(BlockId, Long)])] = { assert (statuses != null) val splitsByAddress = new HashMap[BlockManagerId, ListBuffer[(BlockId, Long)]] for ((status, mapId) <- statuses.iterator.zipWithIndex) { @@ -884,7 +900,7 @@ private[spark] object MapOutputTracker extends Logging { val size = status.getSizeForBlock(part) if (size != 0) { splitsByAddress.getOrElseUpdate(status.location, ListBuffer()) += - ((ShuffleBlockId(shuffleId, mapId, part, status.stageAttemptId), size)) + ((ShuffleBlockId(shuffleId, mapId, part, stageAttemptId), size)) } } } diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 8b744356daae..abef7912a3fd 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -2547,6 +2547,9 @@ object SparkContext extends Logging { private[spark] val SPARK_SCHEDULER_POOL = "spark.scheduler.pool" private[spark] val RDD_SCOPE_KEY = "spark.rdd.scope" private[spark] val RDD_SCOPE_NO_OVERRIDE_KEY = "spark.rdd.scope.noOverride" + private[spark] val IS_INDETERMINATE_STAGE = "spark.stage.isIndeterminateStage" + private[spark] val INDETERMINATE_STAGE_ATTEMPT_ID_PREFIX = + "spark.stage.indeterminateStageAttemptId." /** * Executor id for the driver. In earlier versions of Spark, this was ``, but this was diff --git a/core/src/main/scala/org/apache/spark/TaskContext.scala b/core/src/main/scala/org/apache/spark/TaskContext.scala index 959f246f3f9f..47bf7f3b1588 100644 --- a/core/src/main/scala/org/apache/spark/TaskContext.scala +++ b/core/src/main/scala/org/apache/spark/TaskContext.scala @@ -176,6 +176,17 @@ abstract class TaskContext extends Serializable { */ def getLocalProperty(key: String): String + /** + * The indeterminate attempt ID of the stage that this task belongs to, it returns the stage + * attempt number while the stage is not determinate and returns none on the contrary. + */ + def indeterminateStageAttemptId(shuffleId: Int): Option[Int] = { + val id = getLocalProperty(SparkContext.INDETERMINATE_STAGE_ATTEMPT_ID_PREFIX + shuffleId) + if (id != null) { + Some(id.toInt) + } else None + } + @DeveloperApi def taskMetrics(): TaskMetrics 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 8b2461d6b1f9..44400ba62401 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -39,7 +39,7 @@ import org.apache.spark.internal.config import org.apache.spark.internal.config.Tests.TEST_NO_STAGE_RETRY import org.apache.spark.network.util.JavaUtils import org.apache.spark.partial.{ApproximateActionListener, ApproximateEvaluator, PartialResult} -import org.apache.spark.rdd.{DeterministicLevel, RDD, RDDCheckpointData} +import org.apache.spark.rdd.{RDD, RDDCheckpointData} import org.apache.spark.rpc.RpcTimeout import org.apache.spark.storage._ import org.apache.spark.storage.BlockManagerMessages.BlockManagerHeartbeat @@ -149,6 +149,11 @@ private[spark] class DAGScheduler( */ private[scheduler] val shuffleIdToMapStage = new HashMap[Int, ShuffleMapStage] private[scheduler] val jobIdToActiveJob = new HashMap[Int, ActiveJob] + /** + * Mapping from shuffle id to the indeterminate stage attempt id. Only includes the retried + * stages whose output deterministic level is indeterminate. + */ + private[scheduler] val shuffleIdToIndeterminateStageAttemptId = new HashMap[Int, Int] // Stages we need to run whose parents aren't done private[scheduler] val waitingStages = new HashSet[Stage] @@ -1140,12 +1145,29 @@ private[spark] class DAGScheduler( } stage.makeNewStageAttempt(partitionsToCompute.size, taskIdToLocations.values.toSeq) - - // If there are tasks to execute, record the submission time of the stage. Otherwise, - // post the even without the submission time, which indicates that this stage was - // skipped. if (partitionsToCompute.nonEmpty) { + // If there are tasks to execute, record the submission time of the stage. Otherwise, + // post the even without the submission time, which indicates that this stage was + // skipped. stage.latestInfo.submissionTime = Some(clock.getTimeMillis()) + + // While an indeterminate stage retried, the stage attempt id will be used to extend the + // shuffle file in shuffle write task, and then the mapping of shuffle id to indeterminate + // stage id will be used for shuffle reader task. + if (stage.latestInfo.attemptNumber() > 0 && stage.isIndeterminate) { + properties.setProperty(SparkContext.IS_INDETERMINATE_STAGE, "true") + // deal with shuffle writer side property. + stage match { + case sms: ShuffleMapStage => + val stageAttemptId = stage.latestInfo.attemptNumber() + properties.setProperty( + SparkContext.INDETERMINATE_STAGE_ATTEMPT_ID_PREFIX + sms.shuffleDep.shuffleId, + stageAttemptId.toString) + logInfo(s"Set INDETERMINATE_STAGE_ATTEMPT_ID for $stage(shuffleId:" + + s" ${sms.shuffleDep.shuffleId}) to $stageAttemptId") + case _ => + } + } } listenerBus.post(SparkListenerStageSubmitted(stage.latestInfo, properties)) @@ -1233,8 +1255,7 @@ private[spark] class DAGScheduler( logInfo(s"Submitting ${tasks.size} missing tasks from $stage (${stage.rdd}) (first 15 " + s"tasks are for partitions ${tasks.take(15).map(_.partitionId)})") taskScheduler.submitTasks(new TaskSet( - tasks.toArray, stage.id, stage.latestInfo.attemptNumber, jobId, properties, - stage.latestInfo.attemptNumber > 0 && stage.isIndeterminate)) + tasks.toArray, stage.id, stage.latestInfo.attemptNumber, jobId, properties)) } else { // Because we posted SparkListenerStageSubmitted earlier, we should mark // the stage as completed here in case there are no tasks to run 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 a80db00564cf..64f0a060a247 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala @@ -43,11 +43,6 @@ private[spark] sealed trait MapStatus { * necessary for correctness, since block fetchers are allowed to skip zero-size blocks. */ def getSizeForBlock(reduceId: Int): Long - - /** - * The attempt id of the stage which this task belong to. - */ - def stageAttemptId: Int } @@ -61,14 +56,11 @@ 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], - stageAttemptId: Int = 0): MapStatus = { + def apply(loc: BlockManagerId, uncompressedSizes: Array[Long]): MapStatus = { if (uncompressedSizes.length > minPartitionsToUseHighlyCompressMapStatus) { - HighlyCompressedMapStatus(loc, uncompressedSizes, stageAttemptId) + HighlyCompressedMapStatus(loc, uncompressedSizes) } else { - new CompressedMapStatus(loc, uncompressedSizes, stageAttemptId) + new CompressedMapStatus(loc, uncompressedSizes) } } @@ -108,18 +100,16 @@ 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 stageAtmpId the attempt id of the stage which this task belong to. */ private[spark] class CompressedMapStatus( private[this] var loc: BlockManagerId, - private[this] var compressedSizes: Array[Byte], - private[this] var stageAtmpId: Int) + private[this] var compressedSizes: Array[Byte]) extends MapStatus with Externalizable { - protected def this() = this(null, null.asInstanceOf[Array[Byte]], 0) // For deserialization only + protected def this() = this(null, null.asInstanceOf[Array[Byte]]) // For deserialization only - def this(loc: BlockManagerId, uncompressedSizes: Array[Long], stageAttemptId: Int) { - this(loc, uncompressedSizes.map(MapStatus.compressSize), stageAttemptId) + def this(loc: BlockManagerId, uncompressedSizes: Array[Long]) { + this(loc, uncompressedSizes.map(MapStatus.compressSize)) } override def location: BlockManagerId = loc @@ -132,7 +122,6 @@ private[spark] class CompressedMapStatus( loc.writeExternal(out) out.writeInt(compressedSizes.length) out.write(compressedSizes) - out.writeInt(stageAtmpId) } override def readExternal(in: ObjectInput): Unit = Utils.tryOrIOException { @@ -140,10 +129,7 @@ private[spark] class CompressedMapStatus( val len = in.readInt() compressedSizes = new Array[Byte](len) in.readFully(compressedSizes) - stageAtmpId = in.readInt() } - - override def stageAttemptId: Int = stageAtmpId } /** @@ -156,27 +142,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 stageAtmpId the attempt id of the stage which this task belong to. */ 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 stageAtmpId: Int) + private[this] var hugeBlockSizes: scala.collection.Map[Int, Byte]) 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, "Average size can only be zero for map stages that produced no output") - protected def this() = this(null, -1, null, -1, null, 0) // For deserialization only + protected def this() = this(null, -1, null, -1, null) // For deserialization only override def location: BlockManagerId = loc - override def stageAttemptId: Int = stageAtmpId - override def getSizeForBlock(reduceId: Int): Long = { assert(hugeBlockSizes != null) if (emptyBlocks.contains(reduceId)) { @@ -198,7 +180,6 @@ private[spark] class HighlyCompressedMapStatus private ( out.writeInt(kv._1) out.writeByte(kv._2) } - out.writeInt(stageAtmpId) } override def readExternal(in: ObjectInput): Unit = Utils.tryOrIOException { @@ -214,14 +195,11 @@ private[spark] class HighlyCompressedMapStatus private ( hugeBlockSizesImpl(block) = size } hugeBlockSizes = hugeBlockSizesImpl - stageAtmpId = in.readInt() } } private[spark] object HighlyCompressedMapStatus { - def apply(loc: BlockManagerId, - uncompressedSizes: Array[Long], - stageAttemptId: Int): HighlyCompressedMapStatus = { + def apply(loc: BlockManagerId, uncompressedSizes: Array[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 @@ -262,6 +240,6 @@ private[spark] object HighlyCompressedMapStatus { emptyBlocks.trim() emptyBlocks.runOptimize() new HighlyCompressedMapStatus(loc, numNonEmptyBlocks, emptyBlocks, avgSize, - hugeBlockSizes, stageAttemptId) + hugeBlockSizes) } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSet.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSet.scala index d13bfd331d96..517c8991aed7 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSet.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSet.scala @@ -28,8 +28,7 @@ private[spark] class TaskSet( val stageId: Int, val stageAttemptId: Int, val priority: Int, - val properties: Properties, - val retryIndeterminateStage: Boolean = false) { + val properties: Properties) { val id: String = stageId + "." + stageAttemptId override def toString: String = "TaskSet " + id 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 c7843710413d..af9b75af02b9 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.shuffleClient, blockManager, - mapOutputTracker.getMapSizesByExecutorId(handle.shuffleId, startPartition, endPartition), + mapOutputTracker.getMapSizesByExecutorId(handle.shuffleId, startPartition, endPartition, + context.indeterminateStageAttemptId(handle.shuffleId)), 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/IndexShuffleBlockResolver.scala b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala index d8b1df8477b1..a8a38e7e6293 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala @@ -54,32 +54,32 @@ private[spark] class IndexShuffleBlockResolver( def getDataFile( shuffleId: Int, mapId: Int, - stageAttemptId: Int): File = { + indeterminateAttemptId: Option[Int] = None): File = { blockManager.diskBlockManager.getFile( - ShuffleDataBlockId(shuffleId, mapId, NOOP_REDUCE_ID, stageAttemptId)) + ShuffleDataBlockId(shuffleId, mapId, NOOP_REDUCE_ID, indeterminateAttemptId)) } private def getIndexFile( shuffleId: Int, mapId: Int, - stageAttemptId: Int): File = { + indeterminateAttemptId: Option[Int] = None): File = { blockManager.diskBlockManager.getFile( - ShuffleIndexBlockId(shuffleId, mapId, NOOP_REDUCE_ID, stageAttemptId)) + ShuffleIndexBlockId(shuffleId, mapId, NOOP_REDUCE_ID, indeterminateAttemptId)) } /** * Remove data file and index file that contain the output data from one map. */ def removeDataByMap( - shuffleId: Int, mapId: Int, stageAttemptId: Int): Unit = { - var file = getDataFile(shuffleId, mapId, stageAttemptId) + shuffleId: Int, mapId: Int, indeterminateAttemptId: Option[Int] = None): Unit = { + var file = getDataFile(shuffleId, mapId, indeterminateAttemptId) if (file.exists()) { if (!file.delete()) { logWarning(s"Error deleting data ${file.getPath()}") } } - file = getIndexFile(shuffleId, mapId, stageAttemptId) + file = getIndexFile(shuffleId, mapId, indeterminateAttemptId) if (file.exists()) { if (!file.delete()) { logWarning(s"Error deleting index ${file.getPath()}") @@ -147,11 +147,11 @@ private[spark] class IndexShuffleBlockResolver( mapId: Int, lengths: Array[Long], dataTmp: File, - stageAttemptId: Int): Unit = { - val indexFile = getIndexFile(shuffleId, mapId, stageAttemptId) + indeterminateAttemptId: Option[Int] = None): Unit = { + val indexFile = getIndexFile(shuffleId, mapId, indeterminateAttemptId) val indexTmp = Utils.tempFileWith(indexFile) try { - val dataFile = getDataFile(shuffleId, mapId, stageAttemptId) + val dataFile = getDataFile(shuffleId, mapId, indeterminateAttemptId) // There is only one IndexShuffleBlockResolver per executor, this synchronization make sure // the following check and rename are atomic. synchronized { @@ -203,7 +203,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, blockId.stageAttemptId) + val indexFile = getIndexFile(blockId.shuffleId, blockId.mapId, blockId.indeterminateAttemptId) // 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 @@ -225,7 +225,7 @@ private[spark] class IndexShuffleBlockResolver( } new FileSegmentManagedBuffer( transportConf, - getDataFile(blockId.shuffleId, blockId.mapId, blockId.stageAttemptId), + getDataFile(blockId.shuffleId, blockId.mapId, blockId.indeterminateAttemptId), 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 2085d44eb9c7..1e39158db84b 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 @@ -77,10 +77,10 @@ private[spark] class SortShuffleManager(conf: SparkConf) extends ShuffleManager } /** - * A mapping from the tuple of shuffle ids and stage attempt ids to the number of mappers - * producing output for those shuffles. + * A mapping from shuffle ids to the tuple of number of mappers producing output and + * indeterminate stage attempt id for those shuffles. */ - private[this] val numMapsForShuffle = new ConcurrentHashMap[(Int, Int), Int]().asScala + private[this] val infoMapsForShuffle = new ConcurrentHashMap[Int, (Int, Option[Int])]() override val shuffleBlockResolver = new IndexShuffleBlockResolver(conf) @@ -130,9 +130,10 @@ private[spark] class SortShuffleManager(conf: SparkConf) extends ShuffleManager mapId: Int, context: TaskContext, metrics: ShuffleWriteMetricsReporter): ShuffleWriter[K, V] = { - numMapsForShuffle.putIfAbsent( - (handle.shuffleId, context.stageAttemptNumber()), - handle.asInstanceOf[BaseShuffleHandle[_, _, _]].numMaps) + infoMapsForShuffle.putIfAbsent( + handle.shuffleId, + (handle.asInstanceOf[BaseShuffleHandle[_, _, _]].numMaps, + context.indeterminateStageAttemptId(handle.shuffleId))) val env = SparkEnv.get handle match { case unsafeShuffleHandle: SerializedShuffleHandle[K @unchecked, V @unchecked] => @@ -161,11 +162,10 @@ private[spark] class SortShuffleManager(conf: SparkConf) extends ShuffleManager /** Remove a shuffle's metadata from the ShuffleManager. */ override def unregisterShuffle(shuffleId: Int): Boolean = { - numMapsForShuffle.filterKeys(_._1 == shuffleId).foreach { - case ((_, stageAttemptId), numMaps) => - numMapsForShuffle.remove((shuffleId, stageAttemptId)) + Option(infoMapsForShuffle.remove(shuffleId)).foreach { + case (numMaps, indeterminateAttemptId) => (0 until numMaps).foreach { mapId => - shuffleBlockResolver.removeDataByMap(shuffleId, mapId, stageAttemptId) + shuffleBlockResolver.removeDataByMap(shuffleId, mapId, indeterminateAttemptId) } } 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 bb6a4b160d73..51c5d3af614f 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 @@ -47,7 +47,7 @@ private[spark] class SortShuffleWriter[K, V, C]( private val writeMetrics = context.taskMetrics().shuffleWriteMetrics - private val stageAttemptId = context.stageAttemptNumber() + private val indeterminateAttemptId = context.indeterminateStageAttemptId(handle.shuffleId) /** Write a bunch of records to this task's output */ override def write(records: Iterator[Product2[K, V]]): Unit = { @@ -66,18 +66,18 @@ 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 output = shuffleBlockResolver.getDataFile(dep.shuffleId, mapId, stageAttemptId) + val output = shuffleBlockResolver.getDataFile(dep.shuffleId, mapId, indeterminateAttemptId) val tmp = Utils.tempFileWith(output) try { val blockId = ShuffleBlockId( dep.shuffleId, mapId, IndexShuffleBlockResolver.NOOP_REDUCE_ID, - stageAttemptId) + indeterminateAttemptId) val partitionLengths = sorter.writePartitionedFile(blockId, tmp) shuffleBlockResolver.writeIndexFileAndCommit( - dep.shuffleId, mapId, partitionLengths, tmp, stageAttemptId) - mapStatus = MapStatus(blockManager.shuffleServerId, partitionLengths, stageAttemptId) + dep.shuffleId, mapId, partitionLengths, tmp, indeterminateAttemptId) + mapStatus = MapStatus(blockManager.shuffleServerId, partitionLengths) } finally { if (tmp.exists() && !tmp.delete()) { logError(s"Error while deleting temp file ${tmp.getAbsolutePath}") 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 2fcf88f0c51f..804276cfc3db 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockId.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockId.scala @@ -58,10 +58,12 @@ case class ShuffleBlockId( shuffleId: Int, mapId: Int, reduceId: Int, - stageAttemptId: Int = 0) + indeterminateAttemptId: Option[Int] = None) extends BlockId { - override def name: String = - "shuffle_" + shuffleId + "_" + mapId + "_" + reduceId + "_" + stageAttemptId + override def name: String = { + val nameStr = "shuffle_" + shuffleId + "_" + mapId + "_" + reduceId + if (indeterminateAttemptId.isEmpty) nameStr else nameStr + "_" + indeterminateAttemptId.get + } } @DeveloperApi @@ -69,10 +71,14 @@ case class ShuffleDataBlockId( shuffleId: Int, mapId: Int, reduceId: Int, - stageAttemptId: Int = 0) + indeterminateAttemptId: Option[Int] = None) extends BlockId { - override def name: String = - "shuffle_" + shuffleId + "_" + mapId + "_" + reduceId + "_" + stageAttemptId + ".data" + override def name: String = { + val nameStr = "shuffle_" + shuffleId + "_" + mapId + "_" + reduceId + val nameStrWithIndeterminateAttempt = + if (indeterminateAttemptId.isEmpty) nameStr else nameStr + "_" + indeterminateAttemptId.get + nameStrWithIndeterminateAttempt + ".data" + } } @DeveloperApi @@ -80,10 +86,14 @@ case class ShuffleIndexBlockId( shuffleId: Int, mapId: Int, reduceId: Int, - stageAttemptId: Int = 0) + indeterminateAttemptId: Option[Int] = None) extends BlockId { - override def name: String = - "shuffle_" + shuffleId + "_" + mapId + "_" + reduceId + "_" + stageAttemptId + ".index" + override def name: String = { + val nameStr = "shuffle_" + shuffleId + "_" + mapId + "_" + reduceId + val nameStrWithIndeterminateAttempt = + if (indeterminateAttemptId.isEmpty) nameStr else nameStr + "_" + indeterminateAttemptId.get + nameStrWithIndeterminateAttempt + ".index" + } } @DeveloperApi @@ -118,14 +128,18 @@ private[spark] case class TestBlockId(id: String) extends BlockId { @DeveloperApi class UnrecognizedBlockId(name: String) - extends SparkException(s"Failed to parse $name into a block ID") + extends SparkException(s"Failed to parse $name into a block ID") @DeveloperApi object BlockId { val RDD = "rdd_([0-9]+)_([0-9]+)".r - val SHUFFLE = "shuffle_([0-9]+)_([0-9]+)_([0-9]+)_([0-9]+)".r - val SHUFFLE_DATA = "shuffle_([0-9]+)_([0-9]+)_([0-9]+)_([0-9]+).data".r - val SHUFFLE_INDEX = "shuffle_([0-9]+)_([0-9]+)_([0-9]+)_([0-9]+).index".r + val SHUFFLE = "shuffle_([0-9]+)_([0-9]+)_([0-9]+)".r + val SHUFFLE_DATA = "shuffle_([0-9]+)_([0-9]+)_([0-9]+).data".r + val SHUFFLE_INDEX = "shuffle_([0-9]+)_([0-9]+)_([0-9]+).index".r + // the extend shuffle/data/index is only used when INDETERMINATE stage rerun + val EXTEND_SHUFFLE = "shuffle_([0-9]+)_([0-9]+)_([0-9]+)_([0-9]+)".r + val EXTEND_SHUFFLE_DATA = "shuffle_([0-9]+)_([0-9]+)_([0-9]+)_([0-9]+).data".r + val EXTEND_SHUFFLE_INDEX = "shuffle_([0-9]+)_([0-9]+)_([0-9]+)_([0-9]+).index".r val BROADCAST = "broadcast_([0-9]+)([_A-Za-z0-9]*)".r val TASKRESULT = "taskresult_([0-9]+)".r val STREAM = "input-([0-9]+)-([0-9]+)".r @@ -136,12 +150,18 @@ object BlockId { def apply(name: String): BlockId = name match { case RDD(rddId, splitIndex) => RDDBlockId(rddId.toInt, splitIndex.toInt) - case SHUFFLE(shuffleId, mapId, reduceId, stageAttemptId) => - ShuffleBlockId(shuffleId.toInt, mapId.toInt, reduceId.toInt, stageAttemptId.toInt) - case SHUFFLE_DATA(shuffleId, mapId, reduceId, stageAttemptId) => - ShuffleDataBlockId(shuffleId.toInt, mapId.toInt, reduceId.toInt, stageAttemptId.toInt) - case SHUFFLE_INDEX(shuffleId, mapId, reduceId, stageAttemptId) => - ShuffleIndexBlockId(shuffleId.toInt, mapId.toInt, reduceId.toInt, stageAttemptId.toInt) + case SHUFFLE(shuffleId, mapId, reduceId) => + ShuffleBlockId(shuffleId.toInt, mapId.toInt, reduceId.toInt) + case SHUFFLE_DATA(shuffleId, mapId, reduceId) => + ShuffleDataBlockId(shuffleId.toInt, mapId.toInt, reduceId.toInt) + case SHUFFLE_INDEX(shuffleId, mapId, reduceId) => + ShuffleIndexBlockId(shuffleId.toInt, mapId.toInt, reduceId.toInt) + case EXTEND_SHUFFLE(shuffleId, mapId, reduceId, stageAttemptId) => + ShuffleBlockId(shuffleId.toInt, mapId.toInt, reduceId.toInt, Some(stageAttemptId.toInt)) + case EXTEND_SHUFFLE_DATA(shuffleId, mapId, reduceId, stageAttemptId) => + ShuffleDataBlockId(shuffleId.toInt, mapId.toInt, reduceId.toInt, Some(stageAttemptId.toInt)) + case EXTEND_SHUFFLE_INDEX(shuffleId, mapId, reduceId, stageAttemptId) => + ShuffleIndexBlockId(shuffleId.toInt, mapId.toInt, reduceId.toInt, Some(stageAttemptId.toInt)) case BROADCAST(broadcastId, field) => BroadcastBlockId(broadcastId.toLong, field.stripPrefix("_")) case TASKRESULT(taskId) => 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 96260823d7f3..b6405b4bbbee 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 @@ -132,7 +132,7 @@ public void setUp() throws IOException { }); when(shuffleBlockResolver.getDataFile( - anyInt(), anyInt(), anyInt())).thenReturn(mergedOutputFile); + anyInt(), anyInt(), any())).thenReturn(mergedOutputFile); doAnswer(invocationOnMock -> { partitionSizesInMergedFile = (long[]) invocationOnMock.getArguments()[2]; File tmp = (File) invocationOnMock.getArguments()[3]; @@ -141,7 +141,7 @@ public void setUp() throws IOException { return null; }).when(shuffleBlockResolver) .writeIndexFileAndCommit( - anyInt(), anyInt(), any(long[].class), any(File.class), any(int.class)); + anyInt(), anyInt(), any(long[].class), any(File.class), any(Option.class)); when(diskBlockManager.createTempShuffleBlock()).thenAnswer(invocationOnMock -> { TempShuffleBlockId blockId = new TempShuffleBlockId(UUID.randomUUID()); diff --git a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala index 7cf6d3e431a0..d86975964b55 100644 --- a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala @@ -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), 0)) + BlockManagerId("999", "mps", 1000), Array.fill[Long](4000000)(0))) } val senderAddress = RpcAddress("localhost", 12345) val rpcCallContext = mock(classOf[RpcCallContext]) diff --git a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala index 32678ab25a84..8b1084a8edc7 100644 --- a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala +++ b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala @@ -412,28 +412,6 @@ abstract class ShuffleSuite extends SparkFunSuite with Matchers with LocalSparkC manager.unregisterShuffle(0) } - - test("[SPARK-25341] check local shuffle with stage attempt id for indeterminate map stage") { - sc = new SparkContext("local", "test", conf.clone()) - // Repeated repartition operation will got an indeterminate map stage of shuffle 0. - val rdd = sc.parallelize(1 to 10, 2).repartition(4).repartition(5) - - // Cannot find one of the local shuffle blocks. - val dataFile = sc.env.blockManager.diskBlockManager.getFile( - new ShuffleDataBlockId(0, 0, 0, 0)) - val indexFile = sc.env.blockManager.diskBlockManager.getFile( - new ShuffleIndexBlockId(0, 0, 0, 0)) - assert(!dataFile.exists() && !indexFile.exists()) - - rdd.count() - - // Can find one of the local shuffle blocks. - val dataExistsFile = sc.env.blockManager.diskBlockManager.getFile( - new ShuffleDataBlockId(0, 0, 0, 0)) - val indexExistsFile = sc.env.blockManager.diskBlockManager.getFile( - new ShuffleIndexBlockId(0, 0, 0, 0)) - assert(dataExistsFile.exists() && indexExistsFile.exists()) - } } /** diff --git a/core/src/test/scala/org/apache/spark/deploy/ExternalShuffleServiceDbSuite.scala b/core/src/test/scala/org/apache/spark/deploy/ExternalShuffleServiceDbSuite.scala index bd77366d8d9b..79c7368f0bdb 100644 --- a/core/src/test/scala/org/apache/spark/deploy/ExternalShuffleServiceDbSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/ExternalShuffleServiceDbSuite.scala @@ -58,7 +58,7 @@ class ExternalShuffleServiceDbSuite extends SparkFunSuite { // Write some sort data. dataContext.insertSortShuffleData(0, 0, Array[Array[Byte]](sortBlock0.getBytes(StandardCharsets.UTF_8), - sortBlock1.getBytes(StandardCharsets.UTF_8))) + sortBlock1.getBytes(StandardCharsets.UTF_8)), false) registerExecutor() } @@ -100,7 +100,7 @@ class ExternalShuffleServiceDbSuite extends SparkFunSuite { blockHandler = externalShuffleService.getBlockHandler blockResolver = blockHandler.getBlockResolver - val block0Stream = blockResolver.getBlockData("app0", "exec0", 0, 0, 0, 0).createInputStream + val block0Stream = blockResolver.getBlockData("app0", "exec0", 0, 0, 0).createInputStream val block0 = CharStreams.toString(new InputStreamReader(block0Stream, StandardCharsets.UTF_8)) block0Stream.close() assert(sortBlock0 == block0) @@ -127,7 +127,7 @@ class ExternalShuffleServiceDbSuite extends SparkFunSuite { blockResolver = blockHandler.getBlockResolver val error = intercept[RuntimeException] { - blockResolver.getBlockData("app0", "exec0", 0, 0, 0, 0).createInputStream + blockResolver.getBlockData("app0", "exec0", 0, 0, 0).createInputStream }.getMessage assert(error.contains("not registered")) 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 ff8badeda9ec..ab057fee6081 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -2711,7 +2711,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi val shuffleId2 = shuffleDep2.shuffleId val finalRdd = new MyRDD(sc, 2, List(shuffleDep2), tracker = mapOutputTracker) - submit(finalRdd, Array(0, 1)) + submit(finalRdd, Array(0, 1), properties = new Properties()) // Finish the first shuffle map stage. complete(taskSets(0), Seq( @@ -2765,11 +2765,15 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi (Success, makeMapStatus("hostA", 2)), (Success, makeMapStatus("hostB", 2)))) assert(mapOutputTracker.findMissingPartitions(shuffleId1) === Some(Seq.empty)) + assert(taskSets(4).tasks.head.localProperties.getProperty( + SparkContext.INDETERMINATE_STAGE_ATTEMPT_ID_PREFIX + shuffleId1.toString) == "1") complete(taskSets(5), Seq( (Success, makeMapStatus("hostC", 2)), (Success, makeMapStatus("hostD", 2)))) assert(mapOutputTracker.findMissingPartitions(shuffleId2) === Some(Seq.empty)) + assert(taskSets(5).tasks.head.localProperties.getProperty( + SparkContext.INDETERMINATE_STAGE_ATTEMPT_ID_PREFIX + shuffleId2.toString) == "2") complete(taskSets(6), Seq((Success, 11), (Success, 12))) @@ -2796,7 +2800,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi val shuffleId3 = shuffleDep3.shuffleId val finalRdd = new MyRDD(sc, 2, List(shuffleDep3), tracker = mapOutputTracker) - submit(finalRdd, Array(0, 1)) + submit(finalRdd, Array(0, 1), properties = new Properties()) // Finish the first 3 shuffle map stages. complete(taskSets(0), Seq( @@ -2819,7 +2823,10 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi assert(scheduler.failedStages.toSeq.map(_.id) == Seq(1, 2)) scheduler.resubmitFailedStages() - def checkAndCompleteRetryStage(taskSetIndex: Int, stageId: Int, shuffleId: Int): Unit = { + 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) @@ -2827,6 +2834,8 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi (Success, makeMapStatus("hostA", 2)), (Success, makeMapStatus("hostB", 2)))) assert(mapOutputTracker.findMissingPartitions(shuffleId) === Some(Seq.empty)) + assert(taskSets(taskSetIndex).tasks.head.localProperties.getProperty( + SparkContext.INDETERMINATE_STAGE_ATTEMPT_ID_PREFIX + shuffleId.toString) == "1") } // Check all indeterminate stage roll back. 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 58aa0f10feeb..c1e7fb9a1db1 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/MapStatusSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/MapStatusSuite.scala @@ -87,7 +87,7 @@ 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, 1) + val status = MapStatus(loc, sizes) val status1 = compressAndDecompressMapStatus(status) assert(status1.isInstanceOf[HighlyCompressedMapStatus]) assert(status1.location == loc) @@ -97,7 +97,6 @@ class MapStatusSuite extends SparkFunSuite { assert(estimate === avg) } } - assert(status1.stageAttemptId == 1) } test("SPARK-22540: ensure HighlyCompressedMapStatus calculates correct avgSize") { @@ -110,7 +109,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, 1) + val status = MapStatus(loc, sizes) val status1 = compressAndDecompressMapStatus(status) assert(status1.isInstanceOf[HighlyCompressedMapStatus]) assert(status1.location == loc) @@ -120,7 +119,6 @@ class MapStatusSuite extends SparkFunSuite { assert(estimate === avg) } } - assert(status1.stageAttemptId == 1) } def compressAndDecompressMapStatus(status: MapStatus): MapStatus = { 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 e012597cb184..c6d0e0a74cc5 100644 --- a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala @@ -352,7 +352,7 @@ class KryoSerializerSuite extends SparkFunSuite with SharedSparkContext { val sparseBlockSizes = Array[Long](0L, 1L, 0L, 2L) Seq(denseBlockSizes, sparseBlockSizes).foreach { blockSizes => ser.serialize(HighlyCompressedMapStatus( - BlockManagerId("exec-1", "host", 1234), blockSizes, 0)) + BlockManagerId("exec-1", "host", 1234), blockSizes)) } } 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 6d2ef17a7a79..f78e43c5ebf0 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, None)).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/BypassMergeSortShuffleWriterSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala index f6bc4d8674cf..5dc35b85cc93 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 @@ -67,31 +67,18 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte when(dependency.partitioner).thenReturn(new HashPartitioner(7)) when(dependency.serializer).thenReturn(new JavaSerializer(conf)) when(taskContext.taskMetrics()).thenReturn(taskMetrics) -<<<<<<< HEAD - when(taskContext.getIndeterminateAttemptId).thenReturn(None) + when(taskContext.indeterminateStageAttemptId(any[Int])).thenReturn(None) when(blockResolver.getDataFile(0, 0, None)).thenReturn(outputFile) doAnswer { (invocationOnMock: InvocationOnMock) => val tmp = invocationOnMock.getArguments()(3).asInstanceOf[File] if (tmp != null) { outputFile.delete tmp.renameTo(outputFile) -======= - when(taskContext.stageAttemptNumber()).thenReturn(0) - when(blockResolver.getDataFile(0, 0, 0)).thenReturn(outputFile) - doAnswer(new Answer[Void] { - def answer(invocationOnMock: InvocationOnMock): Void = { - val tmp: File = invocationOnMock.getArguments()(3).asInstanceOf[File] - if (tmp != null) { - outputFile.delete - tmp.renameTo(outputFile) - } - null ->>>>>>> extend all shuffle block id, not only the indeterminate ones. } null }.when(blockResolver) .writeIndexFileAndCommit( - anyInt, anyInt, any(classOf[Array[Long]]), any(classOf[File]), any(classOf[Int])) + anyInt, anyInt, any(classOf[Array[Long]]), any(classOf[File]), any(classOf[Option[Int]])) when(blockManager.diskBlockManager).thenReturn(diskBlockManager) when(blockManager.getDiskWriter( any[BlockId], 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 79941d6878cb..363cd30fd53f 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 @@ -59,7 +59,7 @@ class IndexShuffleBlockResolverSuite extends SparkFunSuite with BeforeAndAfterEa } private def testWithIndexShuffleBlockResolver( - shuffleId: Int, mapId: Int, idxName: String, attemptId: Int): Unit = { + shuffleId: Int, mapId: Int, idxName: String, attemptId: Option[Int]): Unit = { val resolver = new IndexShuffleBlockResolver(conf, blockManager) val lengths = Array[Long](10, 0, 20) val dataTmp = File.createTempFile("shuffle", null, tempDir) @@ -157,14 +157,14 @@ 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_0.index" - testWithIndexShuffleBlockResolver(shuffleId, mapId, idxName, 0) + val idxName = s"shuffle_${shuffleId}_${mapId}_0.index" + testWithIndexShuffleBlockResolver(shuffleId, mapId, idxName, None) } test("commit shuffle files with attempt id multiple times") { val shuffleId = 1 val mapId = 2 - val attemptId = 1 + val attemptId = Some(1) val idxName = s"shuffle_${shuffleId}_${mapId}_0_1.index" testWithIndexShuffleBlockResolver(shuffleId, mapId, idxName, attemptId) } 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 e78d0045a635..7bf510795a05 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockIdSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockIdSuite.scala @@ -55,7 +55,7 @@ class BlockIdSuite extends SparkFunSuite { val id = ShuffleBlockId(1, 2, 3) assertSame(id, ShuffleBlockId(1, 2, 3)) assertDifferent(id, ShuffleBlockId(3, 2, 3)) - assert(id.name === "shuffle_1_2_3_0") + assert(id.name === "shuffle_1_2_3") assert(id.asRDDId === None) assert(id.shuffleId === 1) assert(id.mapId === 2) @@ -68,7 +68,7 @@ class BlockIdSuite extends SparkFunSuite { val id = ShuffleDataBlockId(4, 5, 6) assertSame(id, ShuffleDataBlockId(4, 5, 6)) assertDifferent(id, ShuffleDataBlockId(6, 5, 6)) - assert(id.name === "shuffle_4_5_6_0.data") + assert(id.name === "shuffle_4_5_6.data") assert(id.asRDDId === None) assert(id.shuffleId === 4) assert(id.mapId === 5) @@ -81,7 +81,7 @@ class BlockIdSuite extends SparkFunSuite { val id = ShuffleIndexBlockId(7, 8, 9) assertSame(id, ShuffleIndexBlockId(7, 8, 9)) assertDifferent(id, ShuffleIndexBlockId(9, 8, 9)) - assert(id.name === "shuffle_7_8_9_0.index") + assert(id.name === "shuffle_7_8_9.index") assert(id.asRDDId === None) assert(id.shuffleId === 7) assert(id.mapId === 8) @@ -91,43 +91,43 @@ class BlockIdSuite extends SparkFunSuite { } test("extend shuffle") { - val id = ShuffleBlockId(1, 2, 3, 4) - assertSame(id, ShuffleBlockId(1, 2, 3, 4)) - assertDifferent(id, ShuffleBlockId(3, 2, 3, 5)) + val id = ShuffleBlockId(1, 2, 3, Some(4)) + assertSame(id, ShuffleBlockId(1, 2, 3, Some(4))) + assertDifferent(id, ShuffleBlockId(3, 2, 3, Some(5))) assert(id.name === "shuffle_1_2_3_4") assert(id.asRDDId === None) assert(id.shuffleId === 1) assert(id.mapId === 2) assert(id.reduceId === 3) - assert(id.stageAttemptId === 4) + assert(id.indeterminateAttemptId.get === 4) assert(id.isShuffle) assertSame(id, BlockId(id.toString)) } test("extend shuffle data") { - val id = ShuffleDataBlockId(4, 5, 6, 7) - assertSame(id, ShuffleDataBlockId(4, 5, 6, 7)) - assertDifferent(id, ShuffleDataBlockId(6, 5, 6, 8)) + val id = ShuffleDataBlockId(4, 5, 6, Some(7)) + assertSame(id, ShuffleDataBlockId(4, 5, 6, Some(7))) + assertDifferent(id, ShuffleDataBlockId(6, 5, 6, Some(8))) assert(id.name === "shuffle_4_5_6_7.data") assert(id.asRDDId === None) assert(id.shuffleId === 4) assert(id.mapId === 5) assert(id.reduceId === 6) - assert(id.stageAttemptId === 7) + assert(id.indeterminateAttemptId.get === 7) assert(!id.isShuffle) assertSame(id, BlockId(id.toString)) } test("extend shuffle index") { - val id = ShuffleIndexBlockId(7, 8, 9, 10) - assertSame(id, ShuffleIndexBlockId(7, 8, 9, 10)) - assertDifferent(id, ShuffleIndexBlockId(9, 8, 9, 11)) + val id = ShuffleIndexBlockId(7, 8, 9, Some(10)) + assertSame(id, ShuffleIndexBlockId(7, 8, 9, Some(10))) + assertDifferent(id, ShuffleIndexBlockId(9, 8, 9, Some(11))) assert(id.name === "shuffle_7_8_9_10.index") assert(id.asRDDId === None) assert(id.shuffleId === 7) assert(id.mapId === 8) assert(id.reduceId === 9) - assert(id.stageAttemptId === 10) + assert(id.indeterminateAttemptId.get === 10) assert(!id.isShuffle) assertSame(id, BlockId(id.toString)) } From 0a45acc246dbfee4d5f460a6cf8e6705a94e4029 Mon Sep 17 00:00:00 2001 From: Yuanjian Li Date: Thu, 4 Apr 2019 08:39:14 +0800 Subject: [PATCH 12/16] UT fix and comment address --- .../network/shuffle/ExternalShuffleBlockHandler.java | 12 ++++++------ .../scala/org/apache/spark/scheduler/FakeTask.scala | 4 ++-- .../spark/scheduler/TaskSchedulerImplSuite.scala | 7 +++++-- 3 files changed, 13 insertions(+), 10 deletions(-) diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java index 648aa94e928a..e951e8bb7fbf 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java @@ -223,15 +223,13 @@ private class ManagedBufferIterator implements Iterator { ManagedBufferIterator(String appId, String execId, String[] blockIds) { this.appId = appId; this.execId = execId; - String[] blockId0Parts = blockIds[0].split("_"); - checkBlockId(blockId0Parts, blockIds, 0); + String[] blockId0Parts = splitBlockId(blockIds[0]); this.shuffleId = Integer.parseInt(blockId0Parts[1]); mapIdAndReduceIds = new int[2 * blockIds.length]; this.indeterminateAttemptId = (blockId0Parts.length == 5) ? Integer.parseInt(blockId0Parts[4]) : -1; for (int i = 0; i < blockIds.length; i++) { - String[] blockIdParts = blockIds[i].split("_"); - checkBlockId(blockIdParts, blockIds, i); + String[] blockIdParts = splitBlockId(blockIds[i]); if (Integer.parseInt(blockIdParts[1]) != shuffleId) { throw new IllegalArgumentException("Expected shuffleId=" + shuffleId + ", got:" + blockIds[i]); @@ -241,12 +239,14 @@ private class ManagedBufferIterator implements Iterator { } } - private void checkBlockId(String[] blockIdParts, String[] wholeBlockId, int index) { + private String[] splitBlockId(String blockId) { + String[] blockIdParts = blockId.split("_"); if ((blockIdParts.length != 4 && blockIdParts.length != 5) || !blockIdParts[0].equals("shuffle")) { throw new IllegalArgumentException( - "Unexpected shuffle block id format: " + wholeBlockId[index]); + "Unexpected shuffle block id format: " + blockId); } + return blockIdParts; } @Override diff --git a/core/src/test/scala/org/apache/spark/scheduler/FakeTask.scala b/core/src/test/scala/org/apache/spark/scheduler/FakeTask.scala index b29d32f7b35c..d3da1bf75ecf 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/FakeTask.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/FakeTask.scala @@ -57,7 +57,7 @@ object FakeTask { val tasks = Array.tabulate[Task[_]](numTasks) { i => new FakeTask(stageId, i, if (prefLocs.size != 0) prefLocs(i) else Nil) } - new TaskSet(tasks, stageId, stageAttemptId, priority = 0, null) + new TaskSet(tasks, stageId, stageAttemptId, priority = 0, new Properties()) } def createShuffleMapTaskSet( @@ -92,6 +92,6 @@ object FakeTask { val tasks = Array.tabulate[Task[_]](numTasks) { i => new FakeTask(stageId, i, if (prefLocs.size != 0) prefLocs(i) else Nil, isBarrier = true) } - new TaskSet(tasks, stageId, stageAttempId, priority = 0, null) + new TaskSet(tasks, stageId, stageAttempId, priority = 0, new Properties()) } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala index 137ff2bd167a..1f1b5b08e654 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.scheduler import java.nio.ByteBuffer +import java.util.Properties import scala.collection.mutable.HashMap import scala.concurrent.duration._ @@ -196,7 +197,8 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B config.CPUS_PER_TASK.key -> taskCpus.toString) val numFreeCores = 1 val taskSet = new TaskSet( - Array(new NotSerializableFakeTask(1, 0), new NotSerializableFakeTask(0, 1)), 0, 0, 0, null) + Array(new NotSerializableFakeTask(1, 0), new NotSerializableFakeTask(0, 1)), + 0, 0, 0, new Properties()) val multiCoreWorkerOffers = IndexedSeq(new WorkerOffer("executor0", "host0", taskCpus), new WorkerOffer("executor1", "host1", numFreeCores)) taskScheduler.submitTasks(taskSet) @@ -210,7 +212,8 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B // still be processed without error taskScheduler.submitTasks(FakeTask.createTaskSet(1)) val taskSet2 = new TaskSet( - Array(new NotSerializableFakeTask(1, 0), new NotSerializableFakeTask(0, 1)), 1, 0, 0, null) + Array(new NotSerializableFakeTask(1, 0), new NotSerializableFakeTask(0, 1)), + 1, 0, 0, new Properties()) taskScheduler.submitTasks(taskSet2) taskDescriptions = taskScheduler.resourceOffers(multiCoreWorkerOffers).flatten assert(taskDescriptions.map(_.executorId) === Seq("executor0")) From 43f7803d4e86e6ea8b869c269f6e237522c77e01 Mon Sep 17 00:00:00 2001 From: Yuanjian Li Date: Thu, 4 Apr 2019 18:27:30 +0800 Subject: [PATCH 13/16] address comment --- .../shuffle/ExternalShuffleBlockHandler.java | 6 ++--- .../scala/org/apache/spark/SparkContext.scala | 3 +-- .../apache/spark/scheduler/DAGScheduler.scala | 5 ---- .../shuffle/IndexShuffleBlockResolver.scala | 24 +++++++++---------- .../shuffle/sort/SortShuffleManager.scala | 4 ++-- .../shuffle/sort/SortShuffleWriter.scala | 8 +++---- .../org/apache/spark/storage/BlockId.scala | 12 +++++----- .../apache/spark/storage/BlockIdSuite.scala | 6 ++--- 8 files changed, 31 insertions(+), 37 deletions(-) diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java index e951e8bb7fbf..edb6fc21b13d 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java @@ -218,7 +218,7 @@ private class ManagedBufferIterator implements Iterator { private final int shuffleId; // An array containing mapId and reduceId pairs. private final int[] mapIdAndReduceIds; - private final int indeterminateAttemptId; + private final int stageAttemptId; ManagedBufferIterator(String appId, String execId, String[] blockIds) { this.appId = appId; @@ -226,7 +226,7 @@ private class ManagedBufferIterator implements Iterator { String[] blockId0Parts = splitBlockId(blockIds[0]); this.shuffleId = Integer.parseInt(blockId0Parts[1]); mapIdAndReduceIds = new int[2 * blockIds.length]; - this.indeterminateAttemptId = + this.stageAttemptId = (blockId0Parts.length == 5) ? Integer.parseInt(blockId0Parts[4]) : -1; for (int i = 0; i < blockIds.length; i++) { String[] blockIdParts = splitBlockId(blockIds[i]); @@ -257,7 +257,7 @@ public boolean hasNext() { @Override public ManagedBuffer next() { final ManagedBuffer block = blockManager.getBlockData(appId, execId, shuffleId, - mapIdAndReduceIds[index], mapIdAndReduceIds[index + 1], indeterminateAttemptId); + mapIdAndReduceIds[index], mapIdAndReduceIds[index + 1], stageAttemptId); index += 2; metrics.blockTransferRateBytes.mark(block != null ? block.size() : 0); return block; diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index abef7912a3fd..3428748acbe3 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -2548,8 +2548,7 @@ object SparkContext extends Logging { private[spark] val RDD_SCOPE_KEY = "spark.rdd.scope" private[spark] val RDD_SCOPE_NO_OVERRIDE_KEY = "spark.rdd.scope.noOverride" private[spark] val IS_INDETERMINATE_STAGE = "spark.stage.isIndeterminateStage" - private[spark] val INDETERMINATE_STAGE_ATTEMPT_ID_PREFIX = - "spark.stage.indeterminateStageAttemptId." + private[spark] val INDETERMINATE_STAGE_ATTEMPT_ID_PREFIX = "_shuffle_generation_id" /** * Executor id for the driver. In earlier versions of Spark, this was ``, but this was 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 44400ba62401..b698d33c1e67 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -149,11 +149,6 @@ private[spark] class DAGScheduler( */ private[scheduler] val shuffleIdToMapStage = new HashMap[Int, ShuffleMapStage] private[scheduler] val jobIdToActiveJob = new HashMap[Int, ActiveJob] - /** - * Mapping from shuffle id to the indeterminate stage attempt id. Only includes the retried - * stages whose output deterministic level is indeterminate. - */ - private[scheduler] val shuffleIdToIndeterminateStageAttemptId = new HashMap[Int, Int] // Stages we need to run whose parents aren't done private[scheduler] val waitingStages = new HashSet[Stage] 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 a8a38e7e6293..9c2783b90065 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala @@ -54,32 +54,32 @@ private[spark] class IndexShuffleBlockResolver( def getDataFile( shuffleId: Int, mapId: Int, - indeterminateAttemptId: Option[Int] = None): File = { + stageAttemptId: Option[Int] = None): File = { blockManager.diskBlockManager.getFile( - ShuffleDataBlockId(shuffleId, mapId, NOOP_REDUCE_ID, indeterminateAttemptId)) + ShuffleDataBlockId(shuffleId, mapId, NOOP_REDUCE_ID, stageAttemptId)) } private def getIndexFile( shuffleId: Int, mapId: Int, - indeterminateAttemptId: Option[Int] = None): File = { + stageAttemptId: Option[Int] = None): File = { blockManager.diskBlockManager.getFile( - ShuffleIndexBlockId(shuffleId, mapId, NOOP_REDUCE_ID, indeterminateAttemptId)) + ShuffleIndexBlockId(shuffleId, mapId, NOOP_REDUCE_ID, stageAttemptId)) } /** * Remove data file and index file that contain the output data from one map. */ def removeDataByMap( - shuffleId: Int, mapId: Int, indeterminateAttemptId: Option[Int] = None): Unit = { - var file = getDataFile(shuffleId, mapId, indeterminateAttemptId) + shuffleId: Int, mapId: Int, stageAttemptId: Option[Int] = None): Unit = { + var file = getDataFile(shuffleId, mapId, stageAttemptId) if (file.exists()) { if (!file.delete()) { logWarning(s"Error deleting data ${file.getPath()}") } } - file = getIndexFile(shuffleId, mapId, indeterminateAttemptId) + file = getIndexFile(shuffleId, mapId, stageAttemptId) if (file.exists()) { if (!file.delete()) { logWarning(s"Error deleting index ${file.getPath()}") @@ -147,11 +147,11 @@ private[spark] class IndexShuffleBlockResolver( mapId: Int, lengths: Array[Long], dataTmp: File, - indeterminateAttemptId: Option[Int] = None): Unit = { - val indexFile = getIndexFile(shuffleId, mapId, indeterminateAttemptId) + stageAttemptId: Option[Int] = None): Unit = { + val indexFile = getIndexFile(shuffleId, mapId, stageAttemptId) val indexTmp = Utils.tempFileWith(indexFile) try { - val dataFile = getDataFile(shuffleId, mapId, indeterminateAttemptId) + val dataFile = getDataFile(shuffleId, mapId, stageAttemptId) // There is only one IndexShuffleBlockResolver per executor, this synchronization make sure // the following check and rename are atomic. synchronized { @@ -203,7 +203,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, blockId.indeterminateAttemptId) + val indexFile = getIndexFile(blockId.shuffleId, blockId.mapId, blockId.stageAttemptId) // 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 @@ -225,7 +225,7 @@ private[spark] class IndexShuffleBlockResolver( } new FileSegmentManagedBuffer( transportConf, - getDataFile(blockId.shuffleId, blockId.mapId, blockId.indeterminateAttemptId), + getDataFile(blockId.shuffleId, blockId.mapId, blockId.stageAttemptId), 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 1e39158db84b..d6a2278bb5b7 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 @@ -163,9 +163,9 @@ private[spark] class SortShuffleManager(conf: SparkConf) extends ShuffleManager /** Remove a shuffle's metadata from the ShuffleManager. */ override def unregisterShuffle(shuffleId: Int): Boolean = { Option(infoMapsForShuffle.remove(shuffleId)).foreach { - case (numMaps, indeterminateAttemptId) => + case (numMaps, stageAttemptId) => (0 until numMaps).foreach { mapId => - shuffleBlockResolver.removeDataByMap(shuffleId, mapId, indeterminateAttemptId) + shuffleBlockResolver.removeDataByMap(shuffleId, mapId, stageAttemptId) } } 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 51c5d3af614f..3d7c851d0a84 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 @@ -47,7 +47,7 @@ private[spark] class SortShuffleWriter[K, V, C]( private val writeMetrics = context.taskMetrics().shuffleWriteMetrics - private val indeterminateAttemptId = context.indeterminateStageAttemptId(handle.shuffleId) + private val stageAttemptId = context.indeterminateStageAttemptId(handle.shuffleId) /** Write a bunch of records to this task's output */ override def write(records: Iterator[Product2[K, V]]): Unit = { @@ -66,17 +66,17 @@ 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 output = shuffleBlockResolver.getDataFile(dep.shuffleId, mapId, indeterminateAttemptId) + val output = shuffleBlockResolver.getDataFile(dep.shuffleId, mapId, stageAttemptId) val tmp = Utils.tempFileWith(output) try { val blockId = ShuffleBlockId( dep.shuffleId, mapId, IndexShuffleBlockResolver.NOOP_REDUCE_ID, - indeterminateAttemptId) + stageAttemptId) val partitionLengths = sorter.writePartitionedFile(blockId, tmp) shuffleBlockResolver.writeIndexFileAndCommit( - dep.shuffleId, mapId, partitionLengths, tmp, indeterminateAttemptId) + dep.shuffleId, mapId, partitionLengths, tmp, stageAttemptId) mapStatus = MapStatus(blockManager.shuffleServerId, partitionLengths) } finally { if (tmp.exists() && !tmp.delete()) { 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 804276cfc3db..b71a0e086d94 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockId.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockId.scala @@ -58,11 +58,11 @@ case class ShuffleBlockId( shuffleId: Int, mapId: Int, reduceId: Int, - indeterminateAttemptId: Option[Int] = None) + stageAttemptId: Option[Int] = None) extends BlockId { override def name: String = { val nameStr = "shuffle_" + shuffleId + "_" + mapId + "_" + reduceId - if (indeterminateAttemptId.isEmpty) nameStr else nameStr + "_" + indeterminateAttemptId.get + if (stageAttemptId.isEmpty) nameStr else nameStr + "_" + stageAttemptId.get } } @@ -71,12 +71,12 @@ case class ShuffleDataBlockId( shuffleId: Int, mapId: Int, reduceId: Int, - indeterminateAttemptId: Option[Int] = None) + stageAttemptId: Option[Int] = None) extends BlockId { override def name: String = { val nameStr = "shuffle_" + shuffleId + "_" + mapId + "_" + reduceId val nameStrWithIndeterminateAttempt = - if (indeterminateAttemptId.isEmpty) nameStr else nameStr + "_" + indeterminateAttemptId.get + if (stageAttemptId.isEmpty) nameStr else nameStr + "_" + stageAttemptId.get nameStrWithIndeterminateAttempt + ".data" } } @@ -86,12 +86,12 @@ case class ShuffleIndexBlockId( shuffleId: Int, mapId: Int, reduceId: Int, - indeterminateAttemptId: Option[Int] = None) + stageAttemptId: Option[Int] = None) extends BlockId { override def name: String = { val nameStr = "shuffle_" + shuffleId + "_" + mapId + "_" + reduceId val nameStrWithIndeterminateAttempt = - if (indeterminateAttemptId.isEmpty) nameStr else nameStr + "_" + indeterminateAttemptId.get + if (stageAttemptId.isEmpty) nameStr else nameStr + "_" + stageAttemptId.get nameStrWithIndeterminateAttempt + ".index" } } 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 7bf510795a05..dc2daa078ff5 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockIdSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockIdSuite.scala @@ -99,7 +99,7 @@ class BlockIdSuite extends SparkFunSuite { assert(id.shuffleId === 1) assert(id.mapId === 2) assert(id.reduceId === 3) - assert(id.indeterminateAttemptId.get === 4) + assert(id.stageAttemptId.get === 4) assert(id.isShuffle) assertSame(id, BlockId(id.toString)) } @@ -113,7 +113,7 @@ class BlockIdSuite extends SparkFunSuite { assert(id.shuffleId === 4) assert(id.mapId === 5) assert(id.reduceId === 6) - assert(id.indeterminateAttemptId.get === 7) + assert(id.stageAttemptId.get === 7) assert(!id.isShuffle) assertSame(id, BlockId(id.toString)) } @@ -127,7 +127,7 @@ class BlockIdSuite extends SparkFunSuite { assert(id.shuffleId === 7) assert(id.mapId === 8) assert(id.reduceId === 9) - assert(id.indeterminateAttemptId.get === 10) + assert(id.stageAttemptId.get === 10) assert(!id.isShuffle) assertSame(id, BlockId(id.toString)) } From 34fe64f443e89e4560d2d47122ef34ba30b2fcaf Mon Sep 17 00:00:00 2001 From: Yuanjian Li Date: Sun, 7 Apr 2019 02:06:07 +0800 Subject: [PATCH 14/16] address comment 4.7 --- .../shuffle/ExternalShuffleBlockHandler.java | 6 ++--- .../sort/BypassMergeSortShuffleWriter.java | 2 +- .../shuffle/sort/UnsafeShuffleWriter.java | 2 +- .../scala/org/apache/spark/SparkContext.scala | 3 +-- .../scala/org/apache/spark/TaskContext.scala | 6 ++--- .../apache/spark/scheduler/DAGScheduler.scala | 12 ++++++---- .../spark/scheduler/ShuffleMapStage.scala | 10 -------- .../org/apache/spark/scheduler/Stage.scala | 6 ----- .../org/apache/spark/scheduler/TaskSet.scala | 3 ++- .../shuffle/BlockStoreShuffleReader.scala | 2 +- .../shuffle/IndexShuffleBlockResolver.scala | 24 +++++++++---------- .../shuffle/sort/SortShuffleManager.scala | 6 ++--- .../shuffle/sort/SortShuffleWriter.scala | 8 +++---- .../org/apache/spark/storage/BlockId.scala | 24 +++++++++---------- .../spark/scheduler/DAGSchedulerSuite.scala | 6 ++--- .../BypassMergeSortShuffleWriterSuite.scala | 2 +- .../apache/spark/storage/BlockIdSuite.scala | 6 ++--- 17 files changed, 58 insertions(+), 70 deletions(-) diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java index edb6fc21b13d..9ca5c3b761ce 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java @@ -218,7 +218,7 @@ private class ManagedBufferIterator implements Iterator { private final int shuffleId; // An array containing mapId and reduceId pairs. private final int[] mapIdAndReduceIds; - private final int stageAttemptId; + private final int shuffleGenerationId; ManagedBufferIterator(String appId, String execId, String[] blockIds) { this.appId = appId; @@ -226,7 +226,7 @@ private class ManagedBufferIterator implements Iterator { String[] blockId0Parts = splitBlockId(blockIds[0]); this.shuffleId = Integer.parseInt(blockId0Parts[1]); mapIdAndReduceIds = new int[2 * blockIds.length]; - this.stageAttemptId = + this.shuffleGenerationId = (blockId0Parts.length == 5) ? Integer.parseInt(blockId0Parts[4]) : -1; for (int i = 0; i < blockIds.length; i++) { String[] blockIdParts = splitBlockId(blockIds[i]); @@ -257,7 +257,7 @@ public boolean hasNext() { @Override public ManagedBuffer next() { final ManagedBuffer block = blockManager.getBlockData(appId, execId, shuffleId, - mapIdAndReduceIds[index], mapIdAndReduceIds[index + 1], stageAttemptId); + mapIdAndReduceIds[index], mapIdAndReduceIds[index + 1], shuffleGenerationId); index += 2; metrics.blockTransferRateBytes.mark(block != null ? block.size() : 0); return block; 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 ba0bdfe9a7b9..c2ec0bc41ad1 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 @@ -119,7 +119,7 @@ final class BypassMergeSortShuffleWriter extends ShuffleWriter { this.writeMetrics = writeMetrics; this.serializer = dep.serializer(); this.shuffleBlockResolver = shuffleBlockResolver; - this.stageAttemptId = taskContext.indeterminateStageAttemptId(dep.shuffleId()); + this.stageAttemptId = taskContext.getShuffleGenerationId(dep.shuffleId()); } @Override 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 a2c764343d82..45c032545c19 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 @@ -151,7 +151,7 @@ public UnsafeShuffleWriter( this.outputBufferSizeInBytes = (int) (long) sparkConf.get(package$.MODULE$.SHUFFLE_UNSAFE_FILE_OUTPUT_BUFFER_SIZE()) * 1024; open(); - this.stageAttemptId = taskContext.indeterminateStageAttemptId(dep.shuffleId()); + this.stageAttemptId = taskContext.getShuffleGenerationId(dep.shuffleId()); } private void updatePeakMemoryUsed() { diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 3428748acbe3..ef8d8e40fbd5 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -2547,8 +2547,7 @@ object SparkContext extends Logging { private[spark] val SPARK_SCHEDULER_POOL = "spark.scheduler.pool" private[spark] val RDD_SCOPE_KEY = "spark.rdd.scope" private[spark] val RDD_SCOPE_NO_OVERRIDE_KEY = "spark.rdd.scope.noOverride" - private[spark] val IS_INDETERMINATE_STAGE = "spark.stage.isIndeterminateStage" - private[spark] val INDETERMINATE_STAGE_ATTEMPT_ID_PREFIX = "_shuffle_generation_id" + private[spark] val SHUFFLE_GENERATION_ID_PREFIX = "_shuffle_generation_id_" /** * Executor id for the driver. In earlier versions of Spark, this was ``, but this was diff --git a/core/src/main/scala/org/apache/spark/TaskContext.scala b/core/src/main/scala/org/apache/spark/TaskContext.scala index 47bf7f3b1588..e5c93bdf0165 100644 --- a/core/src/main/scala/org/apache/spark/TaskContext.scala +++ b/core/src/main/scala/org/apache/spark/TaskContext.scala @@ -177,11 +177,11 @@ abstract class TaskContext extends Serializable { def getLocalProperty(key: String): String /** - * The indeterminate attempt ID of the stage that this task belongs to, it returns the stage + * The shuffle generation ID of the stage that this task belongs to, it returns the stage * attempt number while the stage is not determinate and returns none on the contrary. */ - def indeterminateStageAttemptId(shuffleId: Int): Option[Int] = { - val id = getLocalProperty(SparkContext.INDETERMINATE_STAGE_ATTEMPT_ID_PREFIX + shuffleId) + def getShuffleGenerationId(shuffleId: Int): Option[Int] = { + val id = getLocalProperty(SparkContext.SHUFFLE_GENERATION_ID_PREFIX + shuffleId) if (id != null) { Some(id.toInt) } else None 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 b698d33c1e67..6f530fb40c9c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1099,7 +1099,11 @@ private[spark] class DAGScheduler( logDebug("submitMissingTasks(" + stage + ")") // Before find missing partition, do the intermediate state clean work first. - stage.clearIntermediateState() + 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() @@ -1150,13 +1154,12 @@ private[spark] class DAGScheduler( // shuffle file in shuffle write task, and then the mapping of shuffle id to indeterminate // stage id will be used for shuffle reader task. if (stage.latestInfo.attemptNumber() > 0 && stage.isIndeterminate) { - properties.setProperty(SparkContext.IS_INDETERMINATE_STAGE, "true") // deal with shuffle writer side property. stage match { case sms: ShuffleMapStage => val stageAttemptId = stage.latestInfo.attemptNumber() properties.setProperty( - SparkContext.INDETERMINATE_STAGE_ATTEMPT_ID_PREFIX + sms.shuffleDep.shuffleId, + SparkContext.SHUFFLE_GENERATION_ID_PREFIX + sms.shuffleDep.shuffleId, stageAttemptId.toString) logInfo(s"Set INDETERMINATE_STAGE_ATTEMPT_ID for $stage(shuffleId:" + s" ${sms.shuffleDep.shuffleId}) to $stageAttemptId") @@ -1250,7 +1253,8 @@ private[spark] class DAGScheduler( logInfo(s"Submitting ${tasks.size} missing tasks from $stage (${stage.rdd}) (first 15 " + s"tasks are for partitions ${tasks.take(15).map(_.partitionId)})") taskScheduler.submitTasks(new TaskSet( - tasks.toArray, stage.id, stage.latestInfo.attemptNumber, jobId, properties)) + tasks.toArray, stage.id, stage.latestInfo.attemptNumber, jobId, properties, + stage.latestInfo.attemptNumber > 0 && stage.isIndeterminate)) } else { // Because we posted SparkListenerStageSubmitted earlier, we should mark // the stage as completed here in case there are no tasks to run diff --git a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapStage.scala b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapStage.scala index 3bc0d302e500..522fc99e73a8 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapStage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapStage.scala @@ -96,14 +96,4 @@ private[spark] class ShuffleMapStage( .findMissingPartitions(shuffleDep.shuffleId) .getOrElse(0 until numPartitions) } - - /** - * Clear the intermediate status including shuffle status and indeterminate attempt id for - * shuffle map stage. - */ - override def clearIntermediateState(): Unit = { - if (isIndeterminate) { - mapOutputTrackerMaster.unregisterAllMapOutput(shuffleDep.shuffleId) - } - } } 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 89018affdb12..b0b1eec93a14 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala @@ -119,10 +119,4 @@ private[scheduler] abstract class Stage( def isIndeterminate: Boolean = rdd.outputDeterministicLevel == DeterministicLevel.INDETERMINATE - - /** - * Clear the intermediate status (e.g. map status, success task, etc) for the stage, currently - * we call the function before indeterminate stage rerunning. - */ - def clearIntermediateState(): Unit = {} } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSet.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSet.scala index 517c8991aed7..728e5454d504 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSet.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSet.scala @@ -28,7 +28,8 @@ private[spark] class TaskSet( val stageId: Int, val stageAttemptId: Int, val priority: Int, - val properties: Properties) { + val properties: Properties, + val indeterminateStage: Boolean = false) { val id: String = stageId + "." + stageAttemptId override def toString: String = "TaskSet " + id 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 af9b75af02b9..e3475ef74de7 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala @@ -48,7 +48,7 @@ private[spark] class BlockStoreShuffleReader[K, C]( blockManager.shuffleClient, blockManager, mapOutputTracker.getMapSizesByExecutorId(handle.shuffleId, startPartition, endPartition, - context.indeterminateStageAttemptId(handle.shuffleId)), + context.getShuffleGenerationId(handle.shuffleId)), 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/IndexShuffleBlockResolver.scala b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala index 9c2783b90065..705c2db64ed2 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala @@ -54,32 +54,32 @@ private[spark] class IndexShuffleBlockResolver( def getDataFile( shuffleId: Int, mapId: Int, - stageAttemptId: Option[Int] = None): File = { + shuffleGenerationId: Option[Int] = None): File = { blockManager.diskBlockManager.getFile( - ShuffleDataBlockId(shuffleId, mapId, NOOP_REDUCE_ID, stageAttemptId)) + ShuffleDataBlockId(shuffleId, mapId, NOOP_REDUCE_ID, shuffleGenerationId)) } private def getIndexFile( shuffleId: Int, mapId: Int, - stageAttemptId: Option[Int] = None): File = { + shuffleGenerationId: Option[Int] = None): File = { blockManager.diskBlockManager.getFile( - ShuffleIndexBlockId(shuffleId, mapId, NOOP_REDUCE_ID, stageAttemptId)) + ShuffleIndexBlockId(shuffleId, mapId, NOOP_REDUCE_ID, shuffleGenerationId)) } /** * Remove data file and index file that contain the output data from one map. */ def removeDataByMap( - shuffleId: Int, mapId: Int, stageAttemptId: Option[Int] = None): Unit = { - var file = getDataFile(shuffleId, mapId, stageAttemptId) + shuffleId: Int, mapId: Int, shuffleGenerationId: Option[Int] = None): Unit = { + var file = getDataFile(shuffleId, mapId, shuffleGenerationId) if (file.exists()) { if (!file.delete()) { logWarning(s"Error deleting data ${file.getPath()}") } } - file = getIndexFile(shuffleId, mapId, stageAttemptId) + file = getIndexFile(shuffleId, mapId, shuffleGenerationId) if (file.exists()) { if (!file.delete()) { logWarning(s"Error deleting index ${file.getPath()}") @@ -147,11 +147,11 @@ private[spark] class IndexShuffleBlockResolver( mapId: Int, lengths: Array[Long], dataTmp: File, - stageAttemptId: Option[Int] = None): Unit = { - val indexFile = getIndexFile(shuffleId, mapId, stageAttemptId) + shuffleGenerationId: Option[Int] = None): Unit = { + val indexFile = getIndexFile(shuffleId, mapId, shuffleGenerationId) val indexTmp = Utils.tempFileWith(indexFile) try { - val dataFile = getDataFile(shuffleId, mapId, stageAttemptId) + val dataFile = getDataFile(shuffleId, mapId, shuffleGenerationId) // There is only one IndexShuffleBlockResolver per executor, this synchronization make sure // the following check and rename are atomic. synchronized { @@ -203,7 +203,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, blockId.stageAttemptId) + val indexFile = getIndexFile(blockId.shuffleId, blockId.mapId, blockId.shuffleGenerationId) // 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 @@ -225,7 +225,7 @@ private[spark] class IndexShuffleBlockResolver( } new FileSegmentManagedBuffer( transportConf, - getDataFile(blockId.shuffleId, blockId.mapId, blockId.stageAttemptId), + getDataFile(blockId.shuffleId, blockId.mapId, blockId.shuffleGenerationId), 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 d6a2278bb5b7..c78a154088a1 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,7 +133,7 @@ private[spark] class SortShuffleManager(conf: SparkConf) extends ShuffleManager infoMapsForShuffle.putIfAbsent( handle.shuffleId, (handle.asInstanceOf[BaseShuffleHandle[_, _, _]].numMaps, - context.indeterminateStageAttemptId(handle.shuffleId))) + context.getShuffleGenerationId(handle.shuffleId))) val env = SparkEnv.get handle match { case unsafeShuffleHandle: SerializedShuffleHandle[K @unchecked, V @unchecked] => @@ -163,9 +163,9 @@ private[spark] class SortShuffleManager(conf: SparkConf) extends ShuffleManager /** Remove a shuffle's metadata from the ShuffleManager. */ override def unregisterShuffle(shuffleId: Int): Boolean = { Option(infoMapsForShuffle.remove(shuffleId)).foreach { - case (numMaps, stageAttemptId) => + case (numMaps, shuffleGenerationId) => (0 until numMaps).foreach { mapId => - shuffleBlockResolver.removeDataByMap(shuffleId, mapId, stageAttemptId) + shuffleBlockResolver.removeDataByMap(shuffleId, mapId, shuffleGenerationId) } } 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 3d7c851d0a84..2f3cd55ca855 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 @@ -47,7 +47,7 @@ private[spark] class SortShuffleWriter[K, V, C]( private val writeMetrics = context.taskMetrics().shuffleWriteMetrics - private val stageAttemptId = context.indeterminateStageAttemptId(handle.shuffleId) + private val shuffleGenerationId = context.getShuffleGenerationId(handle.shuffleId) /** Write a bunch of records to this task's output */ override def write(records: Iterator[Product2[K, V]]): Unit = { @@ -66,17 +66,17 @@ 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 output = shuffleBlockResolver.getDataFile(dep.shuffleId, mapId, stageAttemptId) + val output = shuffleBlockResolver.getDataFile(dep.shuffleId, mapId, shuffleGenerationId) val tmp = Utils.tempFileWith(output) try { val blockId = ShuffleBlockId( dep.shuffleId, mapId, IndexShuffleBlockResolver.NOOP_REDUCE_ID, - stageAttemptId) + shuffleGenerationId) val partitionLengths = sorter.writePartitionedFile(blockId, tmp) shuffleBlockResolver.writeIndexFileAndCommit( - dep.shuffleId, mapId, partitionLengths, tmp, stageAttemptId) + dep.shuffleId, mapId, partitionLengths, tmp, shuffleGenerationId) mapStatus = MapStatus(blockManager.shuffleServerId, partitionLengths) } finally { if (tmp.exists() && !tmp.delete()) { 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 b71a0e086d94..6f36891fff26 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockId.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockId.scala @@ -58,11 +58,11 @@ case class ShuffleBlockId( shuffleId: Int, mapId: Int, reduceId: Int, - stageAttemptId: Option[Int] = None) + shuffleGenerationId: Option[Int] = None) extends BlockId { override def name: String = { val nameStr = "shuffle_" + shuffleId + "_" + mapId + "_" + reduceId - if (stageAttemptId.isEmpty) nameStr else nameStr + "_" + stageAttemptId.get + if (shuffleGenerationId.isEmpty) nameStr else nameStr + "_" + shuffleGenerationId.get } } @@ -71,12 +71,12 @@ case class ShuffleDataBlockId( shuffleId: Int, mapId: Int, reduceId: Int, - stageAttemptId: Option[Int] = None) + shuffleGenerationId: Option[Int] = None) extends BlockId { override def name: String = { val nameStr = "shuffle_" + shuffleId + "_" + mapId + "_" + reduceId val nameStrWithIndeterminateAttempt = - if (stageAttemptId.isEmpty) nameStr else nameStr + "_" + stageAttemptId.get + if (shuffleGenerationId.isEmpty) nameStr else nameStr + "_" + shuffleGenerationId.get nameStrWithIndeterminateAttempt + ".data" } } @@ -86,12 +86,12 @@ case class ShuffleIndexBlockId( shuffleId: Int, mapId: Int, reduceId: Int, - stageAttemptId: Option[Int] = None) + shuffleGenerationId: Option[Int] = None) extends BlockId { override def name: String = { val nameStr = "shuffle_" + shuffleId + "_" + mapId + "_" + reduceId val nameStrWithIndeterminateAttempt = - if (stageAttemptId.isEmpty) nameStr else nameStr + "_" + stageAttemptId.get + if (shuffleGenerationId.isEmpty) nameStr else nameStr + "_" + shuffleGenerationId.get nameStrWithIndeterminateAttempt + ".index" } } @@ -156,12 +156,12 @@ object BlockId { ShuffleDataBlockId(shuffleId.toInt, mapId.toInt, reduceId.toInt) case SHUFFLE_INDEX(shuffleId, mapId, reduceId) => ShuffleIndexBlockId(shuffleId.toInt, mapId.toInt, reduceId.toInt) - case EXTEND_SHUFFLE(shuffleId, mapId, reduceId, stageAttemptId) => - ShuffleBlockId(shuffleId.toInt, mapId.toInt, reduceId.toInt, Some(stageAttemptId.toInt)) - case EXTEND_SHUFFLE_DATA(shuffleId, mapId, reduceId, stageAttemptId) => - ShuffleDataBlockId(shuffleId.toInt, mapId.toInt, reduceId.toInt, Some(stageAttemptId.toInt)) - case EXTEND_SHUFFLE_INDEX(shuffleId, mapId, reduceId, stageAttemptId) => - ShuffleIndexBlockId(shuffleId.toInt, mapId.toInt, reduceId.toInt, Some(stageAttemptId.toInt)) + case EXTEND_SHUFFLE(shuffleId, mapId, reduceId, shuffleGenerationId) => + ShuffleBlockId(shuffleId.toInt, mapId.toInt, reduceId.toInt, Some(shuffleGenerationId.toInt)) + case EXTEND_SHUFFLE_DATA(shuffleId, mapId, reduceId, shuffleGenerationId) => + ShuffleDataBlockId(shuffleId.toInt, mapId.toInt, reduceId.toInt, Some(shuffleGenerationId.toInt)) + case EXTEND_SHUFFLE_INDEX(shuffleId, mapId, reduceId, shuffleGenerationId) => + ShuffleIndexBlockId(shuffleId.toInt, mapId.toInt, reduceId.toInt, Some(shuffleGenerationId.toInt)) case BROADCAST(broadcastId, field) => BroadcastBlockId(broadcastId.toLong, field.stripPrefix("_")) case TASKRESULT(taskId) => 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 ab057fee6081..4d68986bcf70 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -2766,14 +2766,14 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi (Success, makeMapStatus("hostB", 2)))) assert(mapOutputTracker.findMissingPartitions(shuffleId1) === Some(Seq.empty)) assert(taskSets(4).tasks.head.localProperties.getProperty( - SparkContext.INDETERMINATE_STAGE_ATTEMPT_ID_PREFIX + shuffleId1.toString) == "1") + SparkContext.SHUFFLE_GENERATION_ID_PREFIX + shuffleId1.toString) == "1") complete(taskSets(5), Seq( (Success, makeMapStatus("hostC", 2)), (Success, makeMapStatus("hostD", 2)))) assert(mapOutputTracker.findMissingPartitions(shuffleId2) === Some(Seq.empty)) assert(taskSets(5).tasks.head.localProperties.getProperty( - SparkContext.INDETERMINATE_STAGE_ATTEMPT_ID_PREFIX + shuffleId2.toString) == "2") + SparkContext.SHUFFLE_GENERATION_ID_PREFIX + shuffleId2.toString) == "2") complete(taskSets(6), Seq((Success, 11), (Success, 12))) @@ -2835,7 +2835,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi (Success, makeMapStatus("hostB", 2)))) assert(mapOutputTracker.findMissingPartitions(shuffleId) === Some(Seq.empty)) assert(taskSets(taskSetIndex).tasks.head.localProperties.getProperty( - SparkContext.INDETERMINATE_STAGE_ATTEMPT_ID_PREFIX + shuffleId.toString) == "1") + SparkContext.SHUFFLE_GENERATION_ID_PREFIX + shuffleId.toString) == "1") } // Check all indeterminate stage roll back. 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 5dc35b85cc93..3ef59d140645 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 @@ -67,7 +67,7 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte when(dependency.partitioner).thenReturn(new HashPartitioner(7)) when(dependency.serializer).thenReturn(new JavaSerializer(conf)) when(taskContext.taskMetrics()).thenReturn(taskMetrics) - when(taskContext.indeterminateStageAttemptId(any[Int])).thenReturn(None) + when(taskContext.getShuffleGenerationId(any[Int])).thenReturn(None) when(blockResolver.getDataFile(0, 0, None)).thenReturn(outputFile) doAnswer { (invocationOnMock: InvocationOnMock) => val tmp = invocationOnMock.getArguments()(3).asInstanceOf[File] 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 dc2daa078ff5..e1d176ef4e8d 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockIdSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockIdSuite.scala @@ -99,7 +99,7 @@ class BlockIdSuite extends SparkFunSuite { assert(id.shuffleId === 1) assert(id.mapId === 2) assert(id.reduceId === 3) - assert(id.stageAttemptId.get === 4) + assert(id.shuffleGenerationId.get === 4) assert(id.isShuffle) assertSame(id, BlockId(id.toString)) } @@ -113,7 +113,7 @@ class BlockIdSuite extends SparkFunSuite { assert(id.shuffleId === 4) assert(id.mapId === 5) assert(id.reduceId === 6) - assert(id.stageAttemptId.get === 7) + assert(id.shuffleGenerationId.get === 7) assert(!id.isShuffle) assertSame(id, BlockId(id.toString)) } @@ -127,7 +127,7 @@ class BlockIdSuite extends SparkFunSuite { assert(id.shuffleId === 7) assert(id.mapId === 8) assert(id.reduceId === 9) - assert(id.stageAttemptId.get === 10) + assert(id.shuffleGenerationId.get === 10) assert(!id.isShuffle) assertSame(id, BlockId(id.toString)) } From f0b9887a9f0a3a2758a7c4ff28dd1d13fec2830c Mon Sep 17 00:00:00 2001 From: Yuanjian Li Date: Tue, 9 Apr 2019 22:37:46 +0800 Subject: [PATCH 15/16] fix and address rxin comments --- .../main/scala/org/apache/spark/storage/BlockId.scala | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) 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 6f36891fff26..d4e7f09f538a 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockId.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockId.scala @@ -136,7 +136,8 @@ object BlockId { val SHUFFLE = "shuffle_([0-9]+)_([0-9]+)_([0-9]+)".r val SHUFFLE_DATA = "shuffle_([0-9]+)_([0-9]+)_([0-9]+).data".r val SHUFFLE_INDEX = "shuffle_([0-9]+)_([0-9]+)_([0-9]+).index".r - // the extend shuffle/data/index is only used when INDETERMINATE stage rerun + // The EXTEND_SHUFFLE extends SHUFFLE regex with shuffle generation id, it is only used in the + // scenario of rerunning an INDETERMINATE stage. val EXTEND_SHUFFLE = "shuffle_([0-9]+)_([0-9]+)_([0-9]+)_([0-9]+)".r val EXTEND_SHUFFLE_DATA = "shuffle_([0-9]+)_([0-9]+)_([0-9]+)_([0-9]+).data".r val EXTEND_SHUFFLE_INDEX = "shuffle_([0-9]+)_([0-9]+)_([0-9]+)_([0-9]+).index".r @@ -159,9 +160,11 @@ object BlockId { case EXTEND_SHUFFLE(shuffleId, mapId, reduceId, shuffleGenerationId) => ShuffleBlockId(shuffleId.toInt, mapId.toInt, reduceId.toInt, Some(shuffleGenerationId.toInt)) case EXTEND_SHUFFLE_DATA(shuffleId, mapId, reduceId, shuffleGenerationId) => - ShuffleDataBlockId(shuffleId.toInt, mapId.toInt, reduceId.toInt, Some(shuffleGenerationId.toInt)) + ShuffleDataBlockId( + shuffleId.toInt, mapId.toInt, reduceId.toInt, Some(shuffleGenerationId.toInt)) case EXTEND_SHUFFLE_INDEX(shuffleId, mapId, reduceId, shuffleGenerationId) => - ShuffleIndexBlockId(shuffleId.toInt, mapId.toInt, reduceId.toInt, Some(shuffleGenerationId.toInt)) + ShuffleIndexBlockId( + shuffleId.toInt, mapId.toInt, reduceId.toInt, Some(shuffleGenerationId.toInt)) case BROADCAST(broadcastId, field) => BroadcastBlockId(broadcastId.toLong, field.stripPrefix("_")) case TASKRESULT(taskId) => From b8bc6119ee468f1e0371ac946f07f6362cd16f47 Mon Sep 17 00:00:00 2001 From: Yuanjian Li Date: Mon, 22 Apr 2019 19:11:18 +0800 Subject: [PATCH 16/16] address #24359 --- .../main/scala/org/apache/spark/scheduler/DAGScheduler.scala | 3 +-- core/src/main/scala/org/apache/spark/scheduler/TaskSet.scala | 3 +-- 2 files changed, 2 insertions(+), 4 deletions(-) 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 6f530fb40c9c..fe2453a4e9f9 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1253,8 +1253,7 @@ private[spark] class DAGScheduler( logInfo(s"Submitting ${tasks.size} missing tasks from $stage (${stage.rdd}) (first 15 " + s"tasks are for partitions ${tasks.take(15).map(_.partitionId)})") taskScheduler.submitTasks(new TaskSet( - tasks.toArray, stage.id, stage.latestInfo.attemptNumber, jobId, properties, - stage.latestInfo.attemptNumber > 0 && stage.isIndeterminate)) + tasks.toArray, stage.id, stage.latestInfo.attemptNumber, jobId, properties)) } else { // Because we posted SparkListenerStageSubmitted earlier, we should mark // the stage as completed here in case there are no tasks to run diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSet.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSet.scala index 728e5454d504..517c8991aed7 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSet.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSet.scala @@ -28,8 +28,7 @@ private[spark] class TaskSet( val stageId: Int, val stageAttemptId: Int, val priority: Int, - val properties: Properties, - val indeterminateStage: Boolean = false) { + val properties: Properties) { val id: String = stageId + "." + stageAttemptId override def toString: String = "TaskSet " + id