diff options
author | Stephen Haberman <stephen@exigencecorp.com> | 2013-02-26 01:38:50 -0600 |
---|---|---|
committer | Stephen Haberman <stephen@exigencecorp.com> | 2013-02-26 01:38:50 -0600 |
commit | db957e5bd74437b3ecbf20406796e133257ae668 (patch) | |
tree | d645b21169f4d2967f43f62e1c61de1705ba4b6d /core | |
parent | a65aa549ff9fdf205dd2c1cc04a832f4972c6c3c (diff) | |
download | spark-db957e5bd74437b3ecbf20406796e133257ae668.tar.gz spark-db957e5bd74437b3ecbf20406796e133257ae668.tar.bz2 spark-db957e5bd74437b3ecbf20406796e133257ae668.zip |
Fix MapOutputTrackerSuite.
Diffstat (limited to 'core')
-rw-r--r-- | core/src/test/scala/spark/MapOutputTrackerSuite.scala | 6 |
1 files changed, 4 insertions, 2 deletions
diff --git a/core/src/test/scala/spark/MapOutputTrackerSuite.scala b/core/src/test/scala/spark/MapOutputTrackerSuite.scala index b49cd81e23..3abc584b6a 100644 --- a/core/src/test/scala/spark/MapOutputTrackerSuite.scala +++ b/core/src/test/scala/spark/MapOutputTrackerSuite.scala @@ -82,11 +82,13 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { test("remote fetch") { val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", "localhost", 0) val masterTracker = new MapOutputTracker() - masterTracker.trackerActor = actorSystem.actorOf(Props(new MapOutputTrackerActor(masterTracker))) + masterTracker.trackerActor = actorSystem.actorOf( + Props(new MapOutputTrackerActor(masterTracker)), "MapOutputTracker") val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", "localhost", 0) val slaveTracker = new MapOutputTracker() - slaveTracker.trackerActor = slaveSystem.actorFor("akka://spark@localhost:" + boundPort) + slaveTracker.trackerActor = slaveSystem.actorFor( + "akka://spark@localhost:" + boundPort + "/user/MapOutputTracker") masterTracker.registerShuffle(10, 1) masterTracker.incrementGeneration() |