spark-instrumented-optimizer/core/src/main/scala/spark/MapOutputTracker.scala

162 lines
4.9 KiB
Scala
Raw Normal View History

2011-02-27 17:27:12 -05:00
package spark
import java.util.concurrent.ConcurrentHashMap
2011-02-27 22:15:52 -05:00
import scala.actors._
import scala.actors.Actor._
import scala.actors.remote._
import scala.collection.mutable.HashSet
2011-02-27 22:15:52 -05:00
sealed trait MapOutputTrackerMessage
case class GetMapOutputLocations(shuffleId: Int) extends MapOutputTrackerMessage
case object StopMapOutputTracker extends MapOutputTrackerMessage
class MapOutputTrackerActor(serverUris: ConcurrentHashMap[Int, Array[String]])
extends DaemonActor with Logging {
2011-02-27 22:15:52 -05:00
def act() {
val port = System.getProperty("spark.master.port").toInt
2011-02-27 22:15:52 -05:00
RemoteActor.alive(port)
RemoteActor.register('MapOutputTracker, self)
logInfo("Registered actor on port " + port)
loop {
react {
case GetMapOutputLocations(shuffleId: Int) =>
logInfo("Asked to get map output locations for shuffle " + shuffleId)
reply(serverUris.get(shuffleId))
2012-02-10 11:19:53 -05:00
case StopMapOutputTracker =>
reply('OK)
exit()
}
}
2011-02-27 22:15:52 -05:00
}
}
class MapOutputTracker(isMaster: Boolean) extends Logging {
2011-02-27 22:15:52 -05:00
var trackerActor: AbstractActor = null
2011-05-19 14:19:25 -04:00
private var serverUris = new ConcurrentHashMap[Int, Array[String]]
// Incremented every time a fetch fails so that client nodes know to clear
// their cache of map output locations if this happens.
private var generation: Long = 0
private var generationLock = new java.lang.Object
2011-02-27 22:15:52 -05:00
if (isMaster) {
val tracker = new MapOutputTrackerActor(serverUris)
2011-05-19 14:19:25 -04:00
tracker.start()
trackerActor = tracker
} else {
val host = System.getProperty("spark.master.host")
val port = System.getProperty("spark.master.port").toInt
trackerActor = RemoteActor.select(Node(host, port), 'MapOutputTracker)
2011-02-27 22:15:52 -05:00
}
def registerShuffle(shuffleId: Int, numMaps: Int) {
if (serverUris.get(shuffleId) != null) {
throw new IllegalArgumentException("Shuffle ID " + shuffleId + " registered twice")
}
serverUris.put(shuffleId, new Array[String](numMaps))
}
2011-02-27 22:15:52 -05:00
def registerMapOutput(shuffleId: Int, mapId: Int, serverUri: String) {
2011-02-27 17:27:12 -05:00
var array = serverUris.get(shuffleId)
array.synchronized {
array(mapId) = serverUri
}
2011-02-27 17:27:12 -05:00
}
def registerMapOutputs(shuffleId: Int, locs: Array[String]) {
serverUris.put(shuffleId, Array[String]() ++ locs)
}
def unregisterMapOutput(shuffleId: Int, mapId: Int, serverUri: String) {
var array = serverUris.get(shuffleId)
if (array != null) {
array.synchronized {
2012-02-10 11:19:53 -05:00
if (array(mapId) == serverUri) {
array(mapId) = null
2012-02-10 11:19:53 -05:00
}
}
incrementGeneration()
} else {
throw new SparkException("unregisterMapOutput called for nonexistent shuffle ID")
}
}
2011-02-27 17:27:12 -05:00
// Remembers which map output locations are currently being fetched on a worker
val fetching = new HashSet[Int]
// Called on possibly remote nodes to get the server URIs for a given shuffle
2011-02-27 17:27:12 -05:00
def getServerUris(shuffleId: Int): Array[String] = {
val locs = serverUris.get(shuffleId)
if (locs == null) {
logInfo("Don't have map outputs for " + shuffleId + ", fetching them")
fetching.synchronized {
if (fetching.contains(shuffleId)) {
// Someone else is fetching it; wait for them to be done
while (fetching.contains(shuffleId)) {
2012-02-10 11:19:53 -05:00
try {
fetching.wait()
} catch {
case _ =>
}
}
return serverUris.get(shuffleId)
} else {
fetching += shuffleId
}
}
// We won the race to fetch the output locs; do so
logInfo("Doing the fetch; tracker actor = " + trackerActor)
val fetched = (trackerActor !? GetMapOutputLocations(shuffleId)).asInstanceOf[Array[String]]
serverUris.put(shuffleId, fetched)
fetching.synchronized {
fetching -= shuffleId
fetching.notifyAll()
}
return fetched
} else {
return locs
}
2011-02-27 17:27:12 -05:00
}
def getMapOutputUri(serverUri: String, shuffleId: Int, mapId: Int, reduceId: Int): String = {
"%s/shuffle/%s/%s/%s".format(serverUri, shuffleId, mapId, reduceId)
}
def stop() {
trackerActor !? StopMapOutputTracker
serverUris.clear()
trackerActor = null
}
// Called on master to increment the generation number
def incrementGeneration() {
generationLock.synchronized {
generation += 1
}
}
// Called on master or workers to get current generation number
def getGeneration: Long = {
generationLock.synchronized {
return generation
}
}
// Called on workers to update the generation number, potentially clearing old outputs
// because of a fetch failure. (Each Mesos task calls this with the latest generation
// number on the master at the time it was created.)
def updateGeneration(newGen: Long) {
generationLock.synchronized {
if (newGen > generation) {
logInfo("Updating generation to " + newGen + " and clearing cache")
serverUris = new ConcurrentHashMap[Int, Array[String]]
generation = newGen
}
}
}
}