Skip to content

Commit

Permalink
Simplify test + make access to akka frame size more modular
Browse files Browse the repository at this point in the history
  • Loading branch information
andrewor14 committed Mar 15, 2014
1 parent 281d7c9 commit c9b6109
Show file tree
Hide file tree
Showing 3 changed files with 36 additions and 29 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -37,7 +37,7 @@ private[spark] case object StopMapOutputTracker extends MapOutputTrackerMessage

private[spark] class MapOutputTrackerMasterActor(tracker: MapOutputTrackerMaster, conf: SparkConf)
extends Actor with Logging {
val maxAkkaFrameSize = conf.getInt("spark.akka.frameSize", 10) * 1024 * 1024 // MB
val maxAkkaFrameSize = AkkaUtils.maxFrameSize(conf) * 1024 * 1024 // MB

def receive = {
case GetMapOutputStatuses(shuffleId: Int) =>
Expand Down
7 changes: 6 additions & 1 deletion core/src/main/scala/org/apache/spark/util/AkkaUtils.scala
Original file line number Diff line number Diff line change
Expand Up @@ -49,7 +49,7 @@ private[spark] object AkkaUtils extends Logging {

val akkaTimeout = conf.getInt("spark.akka.timeout", 100)

val akkaFrameSize = conf.getInt("spark.akka.frameSize", 10)
val akkaFrameSize = maxFrameSize(conf)
val akkaLogLifecycleEvents = conf.getBoolean("spark.akka.logLifecycleEvents", false)
val lifecycleEvents = if (akkaLogLifecycleEvents) "on" else "off"
if (!akkaLogLifecycleEvents) {
Expand Down Expand Up @@ -121,4 +121,9 @@ private[spark] object AkkaUtils extends Logging {
def lookupTimeout(conf: SparkConf): FiniteDuration = {
Duration.create(conf.get("spark.akka.lookupTimeout", "30").toLong, "seconds")
}

/** Returns the default max frame size for Akka messages in MB. */
def maxFrameSize(conf: SparkConf): Int = {
conf.getInt("spark.akka.frameSize", 10)
}
}
56 changes: 29 additions & 27 deletions core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@ package org.apache.spark
import scala.concurrent.Await

import akka.actor._
import akka.testkit.TestActorRef
import org.scalatest.FunSuite

import org.apache.spark.scheduler.MapStatus
Expand Down Expand Up @@ -100,7 +101,25 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext {
}

test("remote fetch") {
val (masterTracker, slaveTracker) = setUpMasterSlaveSystem(conf)
val hostname = "localhost"
val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, 0, conf = conf,
securityManager = new SecurityManager(conf))

// Will be cleared by LocalSparkContext
System.setProperty("spark.driver.port", boundPort.toString)

val masterTracker = new MapOutputTrackerMaster(conf)
masterTracker.trackerActor = actorSystem.actorOf(
Props(new MapOutputTrackerMasterActor(masterTracker, conf)), "MapOutputTracker")

val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0, conf = conf,
securityManager = new SecurityManager(conf))
val slaveTracker = new MapOutputTracker(conf)
val selection = slaveSystem.actorSelection(
s"akka.tcp://spark@localhost:$boundPort/user/MapOutputTracker")
val timeout = AkkaUtils.lookupTimeout(conf)
slaveTracker.trackerActor = Await.result(selection.resolveOne(timeout), timeout)

masterTracker.registerShuffle(10, 1)
masterTracker.incrementEpoch()
slaveTracker.updateEpoch(masterTracker.getEpoch)
Expand All @@ -113,7 +132,7 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext {
masterTracker.incrementEpoch()
slaveTracker.updateEpoch(masterTracker.getEpoch)
assert(slaveTracker.getServerStatuses(10, 0).toSeq ===
Seq((BlockManagerId("a", "hostA", 1000, 0), size1000)))
Seq((BlockManagerId("a", "hostA", 1000, 0), size1000)))

masterTracker.unregisterMapOutput(10, 0, BlockManagerId("a", "hostA", 1000, 0))
masterTracker.incrementEpoch()
Expand All @@ -128,42 +147,25 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext {
val newConf = new SparkConf
newConf.set("spark.akka.frameSize", "1")
newConf.set("spark.akka.askTimeout", "1") // Fail fast
val (masterTracker, slaveTracker) = setUpMasterSlaveSystem(newConf)

val masterTracker = new MapOutputTrackerMaster(conf)
val actorSystem = ActorSystem("test")
val actorRef = TestActorRef[MapOutputTrackerMasterActor](
new MapOutputTrackerMasterActor(masterTracker, newConf))(actorSystem)
val masterActor = actorRef.underlyingActor

// Frame size should be ~123B, and no exception should be thrown
masterTracker.registerShuffle(10, 1)
masterTracker.registerMapOutput(10, 0, new MapStatus(
BlockManagerId("88", "mph", 1000, 0), Array.fill[Byte](10)(0)))
slaveTracker.getServerStatuses(10, 0)
masterActor.receive(GetMapOutputStatuses(10))

// Frame size should be ~1.1MB, and MapOutputTrackerMasterActor should throw exception
masterTracker.registerShuffle(20, 100)
(0 until 100).foreach { i =>
masterTracker.registerMapOutput(20, i, new MapStatus(
BlockManagerId("999", "mps", 1000, 0), Array.fill[Byte](4000000)(0)))
}
intercept[SparkException] { slaveTracker.getServerStatuses(20, 0) }
}

private def setUpMasterSlaveSystem(conf: SparkConf) = {
val hostname = "localhost"
val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, 0, conf = conf,
securityManager = new SecurityManager(conf))

// Will be cleared by LocalSparkContext
System.setProperty("spark.driver.port", boundPort.toString)

val masterTracker = new MapOutputTrackerMaster(conf)
masterTracker.trackerActor = actorSystem.actorOf(
Props(new MapOutputTrackerMasterActor(masterTracker, conf)), "MapOutputTracker")

val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0, conf = conf,
securityManager = new SecurityManager(conf))
val slaveTracker = new MapOutputTracker(conf)
val selection = slaveSystem.actorSelection(
s"akka.tcp://spark@localhost:$boundPort/user/MapOutputTracker")
val timeout = AkkaUtils.lookupTimeout(conf)
slaveTracker.trackerActor = Await.result(selection.resolveOne(timeout), timeout)
(masterTracker, slaveTracker)
intercept[SparkException] { masterActor.receive(GetMapOutputStatuses(20)) }
}
}

0 comments on commit c9b6109

Please sign in to comment.