aboutsummaryrefslogtreecommitdiff
path: root/core/src
diff options
context:
space:
mode:
authorLian, Cheng <rhythm.mail@gmail.com>2014-01-06 10:39:05 +0800
committerLian, Cheng <rhythm.mail@gmail.com>2014-01-06 10:39:05 +0800
commit5c152e3e219a44f97d9df38ba00cdc4adbf4d873 (patch)
treedced132e725b53f829eaf4c7732dc0927b83ee96 /core/src
parenta4048ff31e6f8d3e1451d8ae2d5b9edee42cfbbe (diff)
downloadspark-5c152e3e219a44f97d9df38ba00cdc4adbf4d873.tar.gz
spark-5c152e3e219a44f97d9df38ba00cdc4adbf4d873.tar.bz2
spark-5c152e3e219a44f97d9df38ba00cdc4adbf4d873.zip
Fixed several compilation errors in test suites
Diffstat (limited to 'core/src')
-rw-r--r--core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala11
-rw-r--r--core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala2
2 files changed, 8 insertions, 5 deletions
diff --git a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala
index 10b8b441fd..82dc30ecc4 100644
--- a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala
@@ -23,6 +23,7 @@ import akka.actor._
import org.apache.spark.scheduler.MapStatus
import org.apache.spark.storage.BlockManagerId
import org.apache.spark.util.AkkaUtils
+import scala.concurrent.Await
class MapOutputTrackerSuite extends FunSuite with LocalSparkContext {
private val conf = new SparkConf
@@ -101,13 +102,15 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext {
System.setProperty("spark.hostPort", hostname + ":" + boundPort)
val masterTracker = new MapOutputTrackerMaster(conf)
- masterTracker.trackerActor = Left(actorSystem.actorOf(
- Props(new MapOutputTrackerMasterActor(masterTracker)), "MapOutputTracker"))
+ masterTracker.trackerActor = actorSystem.actorOf(
+ Props(new MapOutputTrackerMasterActor(masterTracker)), "MapOutputTracker")
val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0, conf = conf)
val slaveTracker = new MapOutputTracker(conf)
- slaveTracker.trackerActor = Right(slaveSystem.actorSelection(
- "akka.tcp://spark@localhost:" + boundPort + "/user/MapOutputTracker"))
+ 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()
diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala
index a0fc3445be..032c2f2f69 100644
--- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala
@@ -58,7 +58,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT
conf.set("spark.hostPort", "localhost:" + boundPort)
master = new BlockManagerMaster(
- Left(actorSystem.actorOf(Props(new BlockManagerMasterActor(true, conf)))), conf)
+ actorSystem.actorOf(Props(new BlockManagerMasterActor(true, conf))), conf)
// Set the arch to 64-bit and compressedOops to true to get a deterministic test-case
System.setProperty("os.arch", "amd64")