Skip to content

Commit

Permalink
Address some more comments
Browse files Browse the repository at this point in the history
  • Loading branch information
shivaram committed Jun 5, 2015
1 parent 8e31266 commit 9d5831a
Show file tree
Hide file tree
Showing 3 changed files with 19 additions and 14 deletions.
17 changes: 10 additions & 7 deletions core/src/main/scala/org/apache/spark/MapOutputTracker.scala
Original file line number Diff line number Diff line change
Expand Up @@ -310,14 +310,17 @@ private[spark] class MapOutputTrackerMaster(conf: SparkConf)
val ordering = Ordering.by[(BlockManagerId, Long), Long](_._2).reverse
shuffleIdToReduceLocations(shuffleId) = new HashMap[Int, Array[BlockManagerId]]
var r = 0
// HashMap to add up sizes of all blocks at the same location
val locs = new HashMap[BlockManagerId, Long]
while (r < numReducers) {
// Add up sizes of all blocks at the same location
val locs = statuses.map { s =>
(s.location, s.getSizeForBlock(r))
}.groupBy(_._1).mapValues { sizes =>
sizes.map(_._2).reduceLeft(_ + _)
}.toIterator
val topLocs = CollectionUtils.takeOrdered(locs, numTopLocs)(ordering)
var i = 0
locs.clear()
while (i < statuses.length) {
locs(statuses(i).location) = locs.getOrElse(statuses(i).location, 0L) +
statuses(i).getSizeForBlock(r)
i = i + 1
}
val topLocs = CollectionUtils.takeOrdered(locs.toIterator, numTopLocs)(ordering)
shuffleIdToReduceLocations(shuffleId) += (r -> topLocs.map(_._1).toArray)
r = r + 1
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -143,7 +143,9 @@ class DAGScheduler(
private[this] val SHUFFLE_PREF_MAP_THRESHOLD = 1000
// NOTE: This should be less than 2000 as we use HighlyCompressedMapStatus beyond that
private[this] val SHUFFLE_PREF_REDUCE_THRESHOLD = 1000
// Number of preferred locations to use for reducer tasks
// Number of preferred locations to use for reducer tasks.
// Making this smaller will focus on the locations where the most data can be read locally, but
// may lead to more delay in scheduling if all of those locations are busy.
private[scheduler] val NUM_REDUCER_PREF_LOCS = 5

// Called by TaskScheduler to report task's starting.
Expand Down
12 changes: 6 additions & 6 deletions core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala
Original file line number Diff line number Diff line change
Expand Up @@ -212,16 +212,16 @@ class MapOutputTrackerSuite extends SparkFunSuite {
tracker.trackerEndpoint = rpcEnv.setupEndpoint(MapOutputTracker.ENDPOINT_NAME,
new MapOutputTrackerMasterEndpoint(rpcEnv, tracker, conf))
// Setup 3 map tasks
// on hostA with output size 1
// on hostA with output size 1
// on hostB with output size 2
// on hostA with output size 2
// on hostA with output size 2
// on hostB with output size 3
tracker.registerShuffle(10, 3)
tracker.registerMapOutput(10, 0, MapStatus(BlockManagerId("a", "hostA", 1000),
Array(1L)))
Array(2L)))
tracker.registerMapOutput(10, 1, MapStatus(BlockManagerId("a", "hostA", 1000),
Array(1L)))
tracker.registerMapOutput(10, 2, MapStatus(BlockManagerId("b", "hostB", 1000),
Array(2L)))
tracker.registerMapOutput(10, 2, MapStatus(BlockManagerId("b", "hostB", 1000),
Array(3L)))

val topLocs = tracker.getLocationsWithLargestOutputs(10, 0, 1, 1)
assert(topLocs.nonEmpty)
Expand Down

0 comments on commit 9d5831a

Please sign in to comment.