[SPARK-13631][CORE] Thread-safe getLocationsWithLargestOutputs

## What changes were proposed in this pull request?

If a job is being scheduled in one thread which has a dependency on an
RDD currently executing a shuffle in another thread, Spark would throw a
NullPointerException. This patch synchronizes access to `mapStatuses` and
skips null status entries (which are in-progress shuffle tasks).

## How was this patch tested?

Our client code unit test suite, which was reliably reproducing the race
condition with 10 threads, shows that this fixes it. I have not found a minimal
test case to add to Spark, but I will attempt to do so if desired.

The same test case was tripping up on SPARK-4454, which was fixed by
making other DAGScheduler code thread-safe.

shivaram srowen

Author: Andy Sloane <asloane@tetrationanalytics.com>

Closes #11505 from a1k0n/SPARK-13631.
This commit is contained in:
Andy Sloane 2016-03-09 10:25:47 +00:00 committed by Sean Owen
parent 2c5af7d4d9
commit cbff2803ef

View file

@ -376,8 +376,6 @@ private[spark] class MapOutputTrackerMaster(conf: SparkConf)
* @param numReducers total number of reducers in the shuffle
* @param fractionThreshold fraction of total map output size that a location must have
* for it to be considered large.
*
* This method is not thread-safe.
*/
def getLocationsWithLargestOutputs(
shuffleId: Int,
@ -386,8 +384,9 @@ private[spark] class MapOutputTrackerMaster(conf: SparkConf)
fractionThreshold: Double)
: Option[Array[BlockManagerId]] = {
if (mapStatuses.contains(shuffleId)) {
val statuses = mapStatuses(shuffleId)
val statuses = mapStatuses.get(shuffleId).orNull
if (statuses != null) {
statuses.synchronized {
if (statuses.nonEmpty) {
// HashMap to add up sizes of all blocks at the same location
val locs = new HashMap[BlockManagerId, Long]
@ -395,11 +394,17 @@ private[spark] class MapOutputTrackerMaster(conf: SparkConf)
var mapIdx = 0
while (mapIdx < statuses.length) {
val status = statuses(mapIdx)
// status may be null here if we are called between registerShuffle, which creates an
// array with null entries for each output, and registerMapOutputs, which populates it
// with valid status entries. This is possible if one thread schedules a job which
// depends on an RDD which is currently being computed by another thread.
if (status != null) {
val blockSize = status.getSizeForBlock(reducerId)
if (blockSize > 0) {
locs(status.location) = locs.getOrElse(status.location, 0L) + blockSize
totalOutputSize += blockSize
}
}
mapIdx = mapIdx + 1
}
val topLocs = locs.filter { case (loc, size) =>
@ -407,7 +412,8 @@ private[spark] class MapOutputTrackerMaster(conf: SparkConf)
}
// Return if we have any locations which satisfy the required threshold
if (topLocs.nonEmpty) {
return Some(topLocs.map(_._1).toArray)
return Some(topLocs.keys.toArray)
}
}
}
}