-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-2774] - Set preferred locations for reduce tasks #1697
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
3fe76f7
f8390dd
6782dea
3666ff5
acf1a2b
6193039
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -232,6 +232,11 @@ private[spark] class MapOutputTrackerMaster(conf: SparkConf) | |
| protected val mapStatuses = new TimeStampedHashMap[Int, Array[MapStatus]]() | ||
| private val cachedSerializedStatuses = new TimeStampedHashMap[Int, Array[Byte]]() | ||
|
|
||
| // For each shuffleId we also maintain a Map from reducerId -> (location, size) | ||
| // Lazily populated whenever the statuses are requested from DAGScheduler | ||
| private val statusByReducer = | ||
| new TimeStampedHashMap[Int, HashMap[Int, Array[(BlockManagerId, Long)]]]() | ||
|
|
||
| // For cleaning up TimeStampedHashMaps | ||
| private val metadataCleaner = | ||
| new MetadataCleaner(MetadataCleanerType.MAP_OUTPUT_TRACKER, this.cleanup, conf) | ||
|
|
@@ -276,6 +281,7 @@ private[spark] class MapOutputTrackerMaster(conf: SparkConf) | |
| /** Unregister shuffle data */ | ||
| override def unregisterShuffle(shuffleId: Int) { | ||
| mapStatuses.remove(shuffleId) | ||
| statusByReducer.remove(shuffleId) | ||
| cachedSerializedStatuses.remove(shuffleId) | ||
| } | ||
|
|
||
|
|
@@ -284,6 +290,30 @@ private[spark] class MapOutputTrackerMaster(conf: SparkConf) | |
| cachedSerializedStatuses.contains(shuffleId) || mapStatuses.contains(shuffleId) | ||
| } | ||
|
|
||
| // Return the list of locations and blockSizes for each reducer. | ||
| // The map is keyed by reducerId and for each reducer the value contains the array | ||
| // of (location, size) of map outputs. | ||
| // | ||
| // This method is not thread-safe | ||
| def getStatusByReducer(shuffleId: Int): Option[Map[Int, Array[(BlockManagerId, Long)]]] = { | ||
|
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. comment on the thread safety
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. also comment on the semantics of the return value (what does the Int mean - what does the index in the array mean, etc)
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. Added comments -- This method is not thread safe as TimestampedHashMap is not thread safe. However we only call this from DAGScheduler which is single threaded AFAIK |
||
| if (!statusByReducer.contains(shuffleId) && mapStatuses.contains(shuffleId)) { | ||
| val statuses = mapStatuses(shuffleId) | ||
| if (statuses.length > 0) { | ||
| val numReducers = statuses(0).compressedSizes.length | ||
| statusByReducer(shuffleId) = new HashMap[Int, Array[(BlockManagerId, Long)]] | ||
| var r = 0 | ||
| while (r < numReducers) { | ||
| val locs = statuses.map { s => | ||
| (s.location, MapOutputTracker.decompressSize(s.compressedSizes(r))) | ||
| } | ||
| statusByReducer(shuffleId) += (r -> locs) | ||
| r = r + 1 | ||
| } | ||
| } | ||
| } | ||
| statusByReducer.get(shuffleId) | ||
| } | ||
|
|
||
| def incrementEpoch() { | ||
| epochLock.synchronized { | ||
| epoch += 1 | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -39,8 +39,9 @@ import org.apache.spark.executor.TaskMetrics | |
| import org.apache.spark.partial.{ApproximateActionListener, ApproximateEvaluator, PartialResult} | ||
| import org.apache.spark.rdd.RDD | ||
| import org.apache.spark.storage._ | ||
| import org.apache.spark.util.{CallSite, SystemClock, Clock, Utils} | ||
| import org.apache.spark.storage.BlockManagerMessages.BlockManagerHeartbeat | ||
| import org.apache.spark.util.{CallSite, SystemClock, Clock, Utils} | ||
| import org.apache.spark.util.collection.{Utils => CollectionUtils} | ||
|
|
||
| /** | ||
| * The high-level scheduling layer that implements stage-oriented scheduling. It computes a DAG of | ||
|
|
@@ -121,6 +122,9 @@ class DAGScheduler( | |
|
|
||
| private[scheduler] var eventProcessActor: ActorRef = _ | ||
|
|
||
| // Number of preferred locations to use for reducer tasks | ||
| private[scheduler] val NUM_REDUCER_PREF_LOCS = 5 | ||
|
|
||
| private def initializeEventProcessActor() { | ||
| // blocking the thread until supervisor is started, which ensures eventProcessActor is | ||
| // not null before any job is submitted | ||
|
|
@@ -1252,6 +1256,19 @@ class DAGScheduler( | |
| return locs | ||
| } | ||
| } | ||
| case s: 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. add some inline comment explaining this case
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. Done |
||
| // Assign preferred locations for reducers by looking at map output location and sizes | ||
| val mapStatuses = mapOutputTracker.getStatusByReducer(s.shuffleId) | ||
| mapStatuses.map { status => | ||
| // Get the map output locations for this reducer | ||
| if (status.contains(partition)) { | ||
| // Select first few locations as preferred locations for the reducer | ||
| val topLocs = CollectionUtils.takeOrdered(status(partition).iterator, | ||
| NUM_REDUCER_PREF_LOCS)(Ordering.by[(BlockManagerId, Long), Long](_._2).reverse).toSeq | ||
|
|
||
| return topLocs.map(_._1).map(loc => TaskLocation(loc.host, loc.executorId)) | ||
| } | ||
| } | ||
| case _ => | ||
| } | ||
| Nil | ||
|
|
||
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.
should we consider sampling the map tasks to speed up the sort?