aboutsummaryrefslogtreecommitdiff
path: root/core/src/test/scala/org
diff options
context:
space:
mode:
Diffstat (limited to 'core/src/test/scala/org')
-rw-r--r--core/src/test/scala/org/apache/spark/deploy/master/PersistenceEngineSuite.scala13
-rw-r--r--core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtilsSuite.scala3
2 files changed, 10 insertions, 6 deletions
diff --git a/core/src/test/scala/org/apache/spark/deploy/master/PersistenceEngineSuite.scala b/core/src/test/scala/org/apache/spark/deploy/master/PersistenceEngineSuite.scala
index 11e87bd1dd..34775577de 100644
--- a/core/src/test/scala/org/apache/spark/deploy/master/PersistenceEngineSuite.scala
+++ b/core/src/test/scala/org/apache/spark/deploy/master/PersistenceEngineSuite.scala
@@ -73,11 +73,11 @@ class PersistenceEngineSuite extends SparkFunSuite {
assert(persistenceEngine.read[String]("test_").isEmpty)
// Test deserializing objects that contain RpcEndpointRef
- val rpcEnv = RpcEnv.create("test", "localhost", 12345, conf, new SecurityManager(conf))
+ val testRpcEnv = RpcEnv.create("test", "localhost", 12345, conf, new SecurityManager(conf))
try {
// Create a real endpoint so that we can test RpcEndpointRef deserialization
- val workerEndpoint = rpcEnv.setupEndpoint("worker", new RpcEndpoint {
- override val rpcEnv: RpcEnv = rpcEnv
+ val workerEndpoint = testRpcEnv.setupEndpoint("worker", new RpcEndpoint {
+ override val rpcEnv: RpcEnv = testRpcEnv
})
val workerToPersist = new WorkerInfo(
@@ -93,7 +93,8 @@ class PersistenceEngineSuite extends SparkFunSuite {
persistenceEngine.addWorker(workerToPersist)
- val (storedApps, storedDrivers, storedWorkers) = persistenceEngine.readPersistedData(rpcEnv)
+ val (storedApps, storedDrivers, storedWorkers) =
+ persistenceEngine.readPersistedData(testRpcEnv)
assert(storedApps.isEmpty)
assert(storedDrivers.isEmpty)
@@ -110,8 +111,8 @@ class PersistenceEngineSuite extends SparkFunSuite {
assert(workerToPersist.webUiPort === recoveryWorkerInfo.webUiPort)
assert(workerToPersist.publicAddress === recoveryWorkerInfo.publicAddress)
} finally {
- rpcEnv.shutdown()
- rpcEnv.awaitTermination()
+ testRpcEnv.shutdown()
+ testRpcEnv.awaitTermination()
}
}
diff --git a/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtilsSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtilsSuite.scala
index b354914b6f..2eb43b7313 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtilsSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtilsSuite.scala
@@ -17,10 +17,13 @@
package org.apache.spark.scheduler.cluster.mesos
+import scala.language.reflectiveCalls
+
import org.apache.mesos.Protos.Value
import org.mockito.Mockito._
import org.scalatest._
import org.scalatest.mock.MockitoSugar
+
import org.apache.spark.{SparkConf, SparkContext, SparkFunSuite}
class MesosSchedulerUtilsSuite extends SparkFunSuite with Matchers with MockitoSugar {