-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-2774] Set preferred locations for reduce tasks #6652
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 9 commits
3b464b7
34d0283
774751b
bc4dfd6
0171d3c
5093aea
df14cee
ad7cb53
e7d5449
0df3180
8e31266
9d5831a
6cfae98
e5d56bd
77ce7d8
1090b58
68bc29e
f5be578
897a914
2ef2d39
492e25e
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 |
|---|---|---|
|
|
@@ -21,7 +21,7 @@ import java.io._ | |
| import java.util.concurrent.ConcurrentHashMap | ||
| import java.util.zip.{GZIPInputStream, GZIPOutputStream} | ||
|
|
||
| import scala.collection.mutable.{HashSet, Map} | ||
| import scala.collection.mutable.{HashSet, Map, HashMap} | ||
| import scala.collection.JavaConversions._ | ||
| import scala.reflect.ClassTag | ||
|
|
||
|
|
@@ -232,6 +232,12 @@ 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 = | ||
|
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. nit: could you rename this to shuffleIdToReduceLocations? With the current name, I expected the main map to be keyed on reduce id.
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 |
||
| new TimeStampedHashMap[Int, HashMap[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. nit: extra newline
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. Fixed |
||
| // For cleaning up TimeStampedHashMaps | ||
| private val metadataCleaner = | ||
| new MetadataCleaner(MetadataCleanerType.MAP_OUTPUT_TRACKER, this.cleanup, conf) | ||
|
|
@@ -277,13 +283,40 @@ private[spark] class MapOutputTrackerMaster(conf: SparkConf) | |
| override def unregisterShuffle(shuffleId: Int) { | ||
| mapStatuses.remove(shuffleId) | ||
| cachedSerializedStatuses.remove(shuffleId) | ||
| statusByReducer.remove(shuffleId) | ||
| } | ||
|
|
||
| /** Check if the given shuffle is being tracked */ | ||
| def containsShuffle(shuffleId: Int): Boolean = { | ||
| 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 | ||
|
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. Why not use the Scaladoc syntax for this?
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. Now using scaladoc syntax |
||
| def getStatusByReducer( | ||
| shuffleId: Int, | ||
| numReducers: Int) | ||
|
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. As discussed offline, take in a parameter here for maxNumberLocations, and then do the filtering for top 5 on line 309-ish when you create the status method (this avoids some of the scalability issues of creating a gigantic map)
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, what about changing this to something like "getMapLocationsWithBiggestOutputs", and having it also accept the reduceId (and then return Option[Array[(BlockManagerId, Long)]], since the caller always just takes one element out of the map? And then you can just add a comment internally saying that you populate the whole map for use later.
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've moved the @JoshRosen let me know if there is a better way to do this. |
||
| : Option[Map[Int, Array[(BlockManagerId, Long)]]] = { | ||
| if (!statusByReducer.contains(shuffleId) && mapStatuses.contains(shuffleId)) { | ||
| val statuses = mapStatuses(shuffleId) | ||
| if (statuses.length > 0) { | ||
|
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.
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 |
||
| statusByReducer(shuffleId) = new HashMap[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. Also is it possible to combine entries for the same block manager here? I think that, right now, if multiple maps are at the same block manager, they won't be aggregated when thinking about which block manager ID has the most output, so if you have BM 1: 2 bytes right now BM2 will be considered the biggest.
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. Great point. I didn't take this into account before. Now added a test in MapOutputTrackerSuite to test this. |
||
| var r = 0 | ||
| while (r < numReducers) { | ||
| val locs = statuses.map { s => | ||
|
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. If perf. is important here, why not write some java-style code that does an aggregate-by-key using a mutable hashmap?
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. Actually, I think we should just go with a straightforward imperative code here. I don't think it will be significantly longer and can't be slow by accident (it'll also create less garbage).
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. You are right. Its the same 3 lines of code imperatively as well. And create a bit less garbage I reuse the hashmap across reducers now. Let me know if this looks okay |
||
| (s.location, s.getSizeForBlock(r)) | ||
| } | ||
| statusByReducer(shuffleId) += (r -> locs) | ||
| r = r + 1 | ||
| } | ||
| } | ||
| } | ||
| statusByReducer.get(shuffleId) | ||
| } | ||
|
|
||
| def incrementEpoch() { | ||
| epochLock.synchronized { | ||
| epoch += 1 | ||
|
|
@@ -331,6 +364,7 @@ private[spark] class MapOutputTrackerMaster(conf: SparkConf) | |
| private def cleanup(cleanupTime: Long) { | ||
| mapStatuses.clearOldValues(cleanupTime) | ||
| cachedSerializedStatuses.clearOldValues(cleanupTime) | ||
| statusByReducer.clearOldValues(cleanupTime) | ||
| } | ||
| } | ||
|
|
||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -38,6 +38,7 @@ import org.apache.spark.rdd.RDD | |
| import org.apache.spark.storage._ | ||
| import org.apache.spark.unsafe.memory.TaskMemoryManager | ||
| import org.apache.spark.util._ | ||
| import org.apache.spark.util.collection.{Utils => CollectionUtils} | ||
| import org.apache.spark.storage.BlockManagerMessages.BlockManagerHeartbeat | ||
|
|
||
| /** | ||
|
|
@@ -137,6 +138,14 @@ class DAGScheduler( | |
| private[scheduler] val eventProcessLoop = new DAGSchedulerEventProcessLoop(this) | ||
| taskScheduler.setDAGScheduler(this) | ||
|
|
||
| // Number of map, reduce tasks above which we do not assign preferred locations | ||
|
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. Could you add a comment here saying that we limit the size because of scalability issues with sorting the best locations?
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 |
||
| // based on map output sizes. | ||
| private val SHUFFLE_PREF_MAP_THRESHOLD = 1000 | ||
|
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. This could be
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. What's the difference? (Have always thought they were the same!)
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. |
||
| // NOTE: This should be less than 2000 as we use HighlyCompressedMapStatus beyond that | ||
| private val SHUFFLE_PREF_REDUCE_THRESHOLD = 1000 | ||
| // Number of preferred locations to use for reducer tasks | ||
| private[scheduler] val NUM_REDUCER_PREF_LOCS = 5 | ||
|
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. one more comment request here: can you add one sentence explaining this? something like "making this smaller will focus on the locations where the most data can be read locally, but may lead to lower scheduling efficiency or the delay scheduling timers expiring if all of those locations are busy"
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 now |
||
|
|
||
| // Called by TaskScheduler to report task's starting. | ||
| def taskStarted(task: Task[_], taskInfo: TaskInfo) { | ||
| eventProcessLoop.post(BeginEvent(task, taskInfo)) | ||
|
|
@@ -1395,6 +1404,23 @@ 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. Can you add a high-level comment here explaining what this is doing? Something like "For reduce tasks, return the 5 locations with the largest map outputs as preferred locations"
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 -- Also I moved the comment for narrow dependencies close to that. |
||
| if (rdd.partitions.size < SHUFFLE_PREF_REDUCE_THRESHOLD && | ||
| s.rdd.partitions.size < SHUFFLE_PREF_MAP_THRESHOLD) { | ||
| // Assign preferred locations for reducers by looking at map output location and sizes | ||
| val mapStatuses = mapOutputTracker.getStatusByReducer(s.shuffleId, rdd.partitions.size) | ||
| 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 | ||
|
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. It looks like this
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've moved this into MapOutputTracker now and I've pulled it out of the loop there |
||
| return topLocs.map(_._1).map(loc => TaskLocation(loc.host, loc.executorId)) | ||
|
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. It looks like you're
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 is now avoided as we return just one reducer's information from the MapOutputTracker. So we have an |
||
| } | ||
| } | ||
| } | ||
|
|
||
| case _ => | ||
|
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. I think you can delete this pattern now, it shouldn't ever occur. incidentally, this also suggests that It seems that code will break if they are ever given a |
||
| } | ||
| Nil | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -490,8 +490,8 @@ class DAGSchedulerSuite | |
| val reduceRdd = new MyRDD(sc, 2, List(shuffleDep)) | ||
| submit(reduceRdd, Array(0, 1)) | ||
| complete(taskSets(0), Seq( | ||
| (Success, makeMapStatus("hostA", 1)), | ||
| (Success, makeMapStatus("hostB", 1)))) | ||
| (Success, makeMapStatus("hostA", reduceRdd.partitions.size)), | ||
| (Success, makeMapStatus("hostB", reduceRdd.partitions.size)))) | ||
| // the 2nd ResultTask failed | ||
| complete(taskSets(1), Seq( | ||
| (Success, 42), | ||
|
|
@@ -501,7 +501,7 @@ class DAGSchedulerSuite | |
| // ask the scheduler to try it again | ||
| scheduler.resubmitFailedStages() | ||
| // have the 2nd attempt pass | ||
| complete(taskSets(2), Seq((Success, makeMapStatus("hostA", 1)))) | ||
| complete(taskSets(2), Seq((Success, makeMapStatus("hostA", reduceRdd.partitions.size)))) | ||
| // we can see both result blocks now | ||
| assert(mapOutputTracker.getServerStatuses(shuffleId, 0).map(_._1.host) === | ||
| Array("hostA", "hostB")) | ||
|
|
@@ -517,8 +517,8 @@ class DAGSchedulerSuite | |
| val reduceRdd = new MyRDD(sc, 2, List(shuffleDep)) | ||
| submit(reduceRdd, Array(0, 1)) | ||
| complete(taskSets(0), Seq( | ||
| (Success, makeMapStatus("hostA", 1)), | ||
| (Success, makeMapStatus("hostB", 1)))) | ||
| (Success, makeMapStatus("hostA", reduceRdd.partitions.size)), | ||
| (Success, makeMapStatus("hostB", reduceRdd.partitions.size)))) | ||
| // The MapOutputTracker should know about both map output locations. | ||
| assert(mapOutputTracker.getServerStatuses(shuffleId, 0).map(_._1.host) === | ||
| Array("hostA", "hostB")) | ||
|
|
@@ -560,18 +560,18 @@ class DAGSchedulerSuite | |
| assert(newEpoch > oldEpoch) | ||
| val taskSet = taskSets(0) | ||
| // should be ignored for being too old | ||
| runEvent(CompletionEvent( | ||
| taskSet.tasks(0), Success, makeMapStatus("hostA", 1), null, createFakeTaskInfo(), null)) | ||
| runEvent(CompletionEvent(taskSet.tasks(0), Success, makeMapStatus("hostA", | ||
| reduceRdd.partitions.size), null, createFakeTaskInfo(), null)) | ||
| // should work because it's a non-failed host | ||
| runEvent(CompletionEvent( | ||
| taskSet.tasks(0), Success, makeMapStatus("hostB", 1), null, createFakeTaskInfo(), null)) | ||
| runEvent(CompletionEvent(taskSet.tasks(0), Success, makeMapStatus("hostB", | ||
| reduceRdd.partitions.size), null, createFakeTaskInfo(), null)) | ||
| // should be ignored for being too old | ||
| runEvent(CompletionEvent( | ||
| taskSet.tasks(0), Success, makeMapStatus("hostA", 1), null, createFakeTaskInfo(), null)) | ||
| runEvent(CompletionEvent(taskSet.tasks(0), Success, makeMapStatus("hostA", | ||
| reduceRdd.partitions.size), null, createFakeTaskInfo(), null)) | ||
| // should work because it's a new epoch | ||
| taskSet.tasks(1).epoch = newEpoch | ||
| runEvent(CompletionEvent( | ||
| taskSet.tasks(1), Success, makeMapStatus("hostA", 1), null, createFakeTaskInfo(), null)) | ||
| runEvent(CompletionEvent(taskSet.tasks(1), Success, makeMapStatus("hostA", | ||
| reduceRdd.partitions.size), null, createFakeTaskInfo(), null)) | ||
| assert(mapOutputTracker.getServerStatuses(shuffleId, 0).map(_._1) === | ||
| Array(makeBlockManagerId("hostB"), makeBlockManagerId("hostA"))) | ||
| complete(taskSets(1), Seq((Success, 42), (Success, 43))) | ||
|
|
@@ -800,19 +800,63 @@ class DAGSchedulerSuite | |
| assertDataStructuresEmpty() | ||
| } | ||
|
|
||
| test("shuffle with reducer locality") { | ||
|
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. ok last supernit: can you make this name a description of what should happen? "Reduce tasks should be placed locally with map output" and then for the below test "Reduce task locality preferences should only include machine with largest map output" or something?
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 |
||
| // Create an shuffleMapRdd with 1 partition | ||
| val shuffleMapRdd = new MyRDD(sc, 1, Nil) | ||
| val shuffleDep = new ShuffleDependency(shuffleMapRdd, null) | ||
| val shuffleId = shuffleDep.shuffleId | ||
| val reduceRdd = new MyRDD(sc, 1, List(shuffleDep)) | ||
| submit(reduceRdd, Array(0)) | ||
| complete(taskSets(0), Seq( | ||
| (Success, makeMapStatus("hostA", 1)))) | ||
| assert(mapOutputTracker.getServerStatuses(shuffleId, 0).map(_._1) === | ||
| Array(makeBlockManagerId("hostA"))) | ||
|
|
||
| // Reducer should run on the same host that map task ran | ||
| val reduceTaskSet = taskSets(1) | ||
| assertLocations(reduceTaskSet, Seq(Seq("hostA"))) | ||
| complete(reduceTaskSet, Seq((Success, 42))) | ||
| assert(results === Map(0 -> 42)) | ||
| assertDataStructuresEmpty | ||
| } | ||
|
|
||
| test("reducer locality with different sizes") { | ||
| val numMapTasks = scheduler.NUM_REDUCER_PREF_LOCS + 1 | ||
| // Create an shuffleMapRdd with more partitions | ||
| val shuffleMapRdd = new MyRDD(sc, numMapTasks, Nil) | ||
| val shuffleDep = new ShuffleDependency(shuffleMapRdd, null) | ||
| val shuffleId = shuffleDep.shuffleId | ||
| val reduceRdd = new MyRDD(sc, 1, List(shuffleDep)) | ||
| submit(reduceRdd, Array(0)) | ||
|
|
||
| val statuses = (1 to numMapTasks).map { i => | ||
| (Success, makeMapStatus("host" + i, 1, (10*i).toByte)) | ||
| } | ||
| complete(taskSets(0), statuses) | ||
|
|
||
| // Reducer should prefer the last hosts where output size is larger | ||
| val hosts = (1 to numMapTasks).map(i => "host" + i).reverse.take(numMapTasks - 1) | ||
|
|
||
| val reduceTaskSet = taskSets(1) | ||
| assertLocations(reduceTaskSet, Seq(hosts)) | ||
| complete(reduceTaskSet, Seq((Success, 42))) | ||
| assert(results === Map(0 -> 42)) | ||
| assertDataStructuresEmpty | ||
| } | ||
|
|
||
| /** | ||
| * Assert that the supplied TaskSet has exactly the given hosts as its preferred locations. | ||
| * Note that this checks only the host and not the executor ID. | ||
| */ | ||
| private def assertLocations(taskSet: TaskSet, hosts: Seq[Seq[String]]) { | ||
| assert(hosts.size === taskSet.tasks.size) | ||
| for ((taskLocs, expectedLocs) <- taskSet.tasks.map(_.preferredLocations).zip(hosts)) { | ||
| assert(taskLocs.map(_.host) === expectedLocs) | ||
| assert(taskLocs.map(_.host).toSet === expectedLocs.toSet) | ||
| } | ||
| } | ||
|
|
||
| private def makeMapStatus(host: String, reduces: Int): MapStatus = | ||
| MapStatus(makeBlockManagerId(host), Array.fill[Long](reduces)(2)) | ||
| private def makeMapStatus(host: String, reduces: Int, sizes: Byte = 2): MapStatus = | ||
| MapStatus(makeBlockManagerId(host), Array.fill[Long](reduces)(sizes)) | ||
|
|
||
| private def makeBlockManagerId(host: String): BlockManagerId = | ||
| BlockManagerId("exec-" + host, host, 12345) | ||
|
|
||
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.
import ordering
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.
Done