-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-20715] Store MapStatuses only in MapOutputTracker, not ShuffleMapStage #17955
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
9e12f59
52be832
1aa14f8
a4298da
8ed62bd
5683ec1
06ef8d3
e9caad5
54a033c
f4c096f
7d59bbe
e3da298
a8069a3
813433a
4550f61
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Large diffs are not rendered by default.
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -328,25 +328,14 @@ class DAGScheduler( | |
| val numTasks = rdd.partitions.length | ||
| val parents = getOrCreateParentStages(rdd, jobId) | ||
| val id = nextStageId.getAndIncrement() | ||
| val stage = new ShuffleMapStage(id, rdd, numTasks, parents, jobId, rdd.creationSite, shuffleDep) | ||
| val stage = new ShuffleMapStage( | ||
| id, rdd, numTasks, parents, jobId, rdd.creationSite, shuffleDep, mapOutputTracker) | ||
|
|
||
| stageIdToStage(id) = stage | ||
| shuffleIdToMapStage(shuffleDep.shuffleId) = stage | ||
| updateJobIdStageIdMaps(jobId, stage) | ||
|
|
||
| if (mapOutputTracker.containsShuffle(shuffleDep.shuffleId)) { | ||
| // A previously run stage generated partitions for this shuffle, so for each output | ||
| // that's still available, copy information about that output location to the new stage | ||
| // (so we don't unnecessarily re-compute that data). | ||
| val serLocs = mapOutputTracker.getSerializedMapOutputStatuses(shuffleDep.shuffleId) | ||
| val locs = MapOutputTracker.deserializeMapStatuses(serLocs) | ||
| (0 until locs.length).foreach { i => | ||
| if (locs(i) ne null) { | ||
| // locs(i) will be null if missing | ||
| stage.addOutputLoc(i, locs(i)) | ||
| } | ||
| } | ||
| } else { | ||
| if (!mapOutputTracker.containsShuffle(shuffleDep.shuffleId)) { | ||
| // Kind of ugly: need to register RDDs with the cache and map output tracker here | ||
| // since we can't do it in the RDD constructor because # of partitions is unknown | ||
| logInfo("Registering RDD " + rdd.id + " (" + rdd.getCreationSite + ")") | ||
|
|
@@ -1217,7 +1206,8 @@ class DAGScheduler( | |
| // The epoch of the task is acceptable (i.e., the task was launched after the most | ||
| // recent failure we're aware of for the executor), so mark the task's output as | ||
| // available. | ||
| shuffleStage.addOutputLoc(smt.partitionId, status) | ||
|
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. In the old code we'd incrementally register map outputs in |
||
| mapOutputTracker.registerMapOutput( | ||
| shuffleStage.shuffleDep.shuffleId, smt.partitionId, status) | ||
| // Remove the task's partition from pending partitions. This may have already been | ||
| // done above, but will not have been done yet in cases where the task attempt was | ||
| // from an earlier attempt of the stage (i.e., not the attempt that's currently | ||
|
|
@@ -1234,16 +1224,14 @@ class DAGScheduler( | |
| logInfo("waiting: " + waitingStages) | ||
| logInfo("failed: " + failedStages) | ||
|
|
||
| // We supply true to increment the epoch number here in case this is a | ||
| // recomputation of the map outputs. In that case, some nodes may have cached | ||
| // locations with holes (from when we detected the error) and will need the | ||
| // epoch incremented to refetch them. | ||
| // TODO: Only increment the epoch number if this is not the first time | ||
| // we registered these map outputs. | ||
| mapOutputTracker.registerMapOutputs( | ||
| shuffleStage.shuffleDep.shuffleId, | ||
| shuffleStage.outputLocInMapOutputTrackerFormat(), | ||
| changeEpoch = true) | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Is it safer if we increment the epoch number here?
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I need to think about this carefully and maybe make a matrix of possible cases to be sure. My original thought process was something like this:
That said, the increment here is only occurring once per stage completion. It probably doesn't hurt to bump the epoch here because in a single-stage-at-a-time case we'd only be invalidating map outputs which we'll never fetch again anyways. Even if we were unnecessarily invalidating the map output statuses of other concurrent stages I think that the impact of this is going to be relatively small (if we did find that this had an impact then a sane approach would be to implement an e-tag like mechanism where bumping the epoch doesn't purge the executor-side caches, but, instead, has them verify a per-stage epoch / counter). Finally, the existing code might be giving us nice eager cleanup of map statuses after stages complete (vs. the cleanup which occurs later when stages or shuffles are fully cleaned up). I think you're right that this change carries unnecessary / not-fully-understood risks for now, so let me go ahead and put in an explicit increment here (with an updated comment / ref. to this discussion) in my next push to this PR. |
||
| // This call to increment the epoch may not be strictly necessary, but it is retained | ||
| // for now in order to minimize the changes in behavior from an earlier version of the | ||
| // code. This existing behavior of always incrementing the epoch following any | ||
| // successful shuffle map stage completion may have benefits by causing unneeded | ||
| // cached map outputs to be cleaned up earlier on executors. In the future we can | ||
| // consider removing this call, but this will require some extra investigation. | ||
| // See https://github.com/apache/spark/pull/17955/files#r117385673 for more details. | ||
| mapOutputTracker.incrementEpoch() | ||
|
|
||
| clearCacheLocs() | ||
|
|
||
|
|
@@ -1343,7 +1331,6 @@ class DAGScheduler( | |
| } | ||
| // Mark the map whose fetch failed as broken in the map stage | ||
| if (mapId != -1) { | ||
| mapStage.removeOutputLoc(mapId, bmAddress) | ||
| mapOutputTracker.unregisterMapOutput(shuffleId, mapId, bmAddress) | ||
| } | ||
|
|
||
|
|
@@ -1393,17 +1380,7 @@ class DAGScheduler( | |
|
|
||
| if (filesLost || !env.blockManager.externalShuffleServiceEnabled) { | ||
| logInfo("Shuffle files lost for executor: %s (epoch %d)".format(execId, currentEpoch)) | ||
| // TODO: This will be really slow if we keep accumulating shuffle map stages | ||
| for ((shuffleId, stage) <- shuffleIdToMapStage) { | ||
| stage.removeOutputsOnExecutor(execId) | ||
| mapOutputTracker.registerMapOutputs( | ||
| shuffleId, | ||
| stage.outputLocInMapOutputTrackerFormat(), | ||
|
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This was a potential inefficiency in the old code. If you lost a single executor then we'd end up overwriting the While the new code still needs to scan every task it doesn't have to scan a separate list per task and only needs a single scan, not two. |
||
| changeEpoch = true) | ||
| } | ||
| if (shuffleIdToMapStage.isEmpty) { | ||
| mapOutputTracker.incrementEpoch() | ||
|
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I think the idea behind this branch was to ensure that we always increment the |
||
| } | ||
| mapOutputTracker.removeOutputsOnExecutor(execId) | ||
| clearCacheLocs() | ||
| } | ||
| } else { | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -19,9 +19,8 @@ package org.apache.spark.scheduler | |
|
|
||
| import scala.collection.mutable.HashSet | ||
|
|
||
| import org.apache.spark.ShuffleDependency | ||
| import org.apache.spark.{MapOutputTrackerMaster, ShuffleDependency, SparkEnv} | ||
| import org.apache.spark.rdd.RDD | ||
| import org.apache.spark.storage.BlockManagerId | ||
| import org.apache.spark.util.CallSite | ||
|
|
||
| /** | ||
|
|
@@ -42,13 +41,12 @@ private[spark] class ShuffleMapStage( | |
| parents: List[Stage], | ||
| firstJobId: Int, | ||
| callSite: CallSite, | ||
| val shuffleDep: ShuffleDependency[_, _, _]) | ||
| val shuffleDep: ShuffleDependency[_, _, _], | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Seems we can pass the
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Good catch. I agree, but with the caveat that we can only clean this up if this isn't functioning as the last strong reference which keeps the dependency from being garbage-collected.
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I'm actually going to leave this as-is for now since I think the reference might actually be serving a purpose and I want to minimize scope of changes for now. |
||
| mapOutputTrackerMaster: MapOutputTrackerMaster) | ||
| extends Stage(id, rdd, numTasks, parents, firstJobId, callSite) { | ||
|
|
||
| private[this] var _mapStageJobs: List[ActiveJob] = Nil | ||
|
|
||
| private[this] var _numAvailableOutputs: Int = 0 | ||
|
|
||
| /** | ||
| * Partitions that either haven't yet been computed, or that were computed on an executor | ||
| * that has since been lost, so should be re-computed. This variable is used by the | ||
|
|
@@ -60,13 +58,6 @@ private[spark] class ShuffleMapStage( | |
| */ | ||
| val pendingPartitions = new HashSet[Int] | ||
|
|
||
| /** | ||
| * List of [[MapStatus]] for each partition. The index of the array is the map partition id, | ||
| * and each value in the array is the list of possible [[MapStatus]] for a partition | ||
| * (a single task might run multiple times). | ||
| */ | ||
| private[this] val outputLocs = Array.fill[List[MapStatus]](numPartitions)(Nil) | ||
|
|
||
| override def toString: String = "ShuffleMapStage " + id | ||
|
|
||
| /** | ||
|
|
@@ -88,69 +79,18 @@ private[spark] class ShuffleMapStage( | |
| /** | ||
| * Number of partitions that have shuffle outputs. | ||
| * When this reaches [[numPartitions]], this map stage is ready. | ||
| * This should be kept consistent as `outputLocs.filter(!_.isEmpty).size`. | ||
| */ | ||
| def numAvailableOutputs: Int = _numAvailableOutputs | ||
| def numAvailableOutputs: Int = mapOutputTrackerMaster.getNumAvailableOutputs(shuffleDep.shuffleId) | ||
|
|
||
| /** | ||
| * Returns true if the map stage is ready, i.e. all partitions have shuffle outputs. | ||
| * This should be the same as `outputLocs.contains(Nil)`. | ||
| */ | ||
| def isAvailable: Boolean = _numAvailableOutputs == numPartitions | ||
| def isAvailable: Boolean = numAvailableOutputs == numPartitions | ||
|
|
||
| /** Returns the sequence of partition ids that are missing (i.e. needs to be computed). */ | ||
| override def findMissingPartitions(): Seq[Int] = { | ||
| val missing = (0 until numPartitions).filter(id => outputLocs(id).isEmpty) | ||
| assert(missing.size == numPartitions - _numAvailableOutputs, | ||
| s"${missing.size} missing, expected ${numPartitions - _numAvailableOutputs}") | ||
| missing | ||
| } | ||
|
|
||
| def addOutputLoc(partition: Int, status: MapStatus): Unit = { | ||
| val prevList = outputLocs(partition) | ||
| outputLocs(partition) = status :: prevList | ||
| if (prevList == Nil) { | ||
| _numAvailableOutputs += 1 | ||
| } | ||
| } | ||
|
|
||
| def removeOutputLoc(partition: Int, bmAddress: BlockManagerId): Unit = { | ||
| val prevList = outputLocs(partition) | ||
| val newList = prevList.filterNot(_.location == bmAddress) | ||
| outputLocs(partition) = newList | ||
| if (prevList != Nil && newList == Nil) { | ||
| _numAvailableOutputs -= 1 | ||
| } | ||
| } | ||
|
|
||
| /** | ||
| * Returns an array of [[MapStatus]] (index by partition id). For each partition, the returned | ||
| * value contains only one (i.e. the first) [[MapStatus]]. If there is no entry for the partition, | ||
| * that position is filled with null. | ||
| */ | ||
| def outputLocInMapOutputTrackerFormat(): Array[MapStatus] = { | ||
| outputLocs.map(_.headOption.orNull) | ||
| } | ||
|
|
||
| /** | ||
| * Removes all shuffle outputs associated with this executor. Note that this will also remove | ||
| * outputs which are served by an external shuffle server (if one exists), as they are still | ||
| * registered with this execId. | ||
| */ | ||
| def removeOutputsOnExecutor(execId: String): Unit = { | ||
| var becameUnavailable = false | ||
| for (partition <- 0 until numPartitions) { | ||
| val prevList = outputLocs(partition) | ||
| val newList = prevList.filterNot(_.location.executorId == execId) | ||
| outputLocs(partition) = newList | ||
| if (prevList != Nil && newList == Nil) { | ||
| becameUnavailable = true | ||
| _numAvailableOutputs -= 1 | ||
| } | ||
| } | ||
| if (becameUnavailable) { | ||
| logInfo("%s is now unavailable on executor %s (%d/%d, %s)".format( | ||
| this, execId, _numAvailableOutputs, numPartitions, isAvailable)) | ||
| } | ||
| mapOutputTrackerMaster | ||
| .findMissingPartitions(shuffleDep.shuffleId) | ||
| .getOrElse(0 until numPartitions) | ||
| } | ||
| } | ||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This logic is no longer necessary because
ShuffleMapTaskqueries theMapOutputTracker.