diff options
author | Charles Reiss <charles@eecs.berkeley.edu> | 2013-03-09 00:05:42 -0800 |
---|---|---|
committer | Charles Reiss <charles@eecs.berkeley.edu> | 2013-03-09 10:49:02 -0800 |
commit | d0216cb38bf976a3e98d1748e1a0ca76c8c7e713 (patch) | |
tree | a0742cd1db3dbeaf0b70c41a55935ca4e16e46cf | |
parent | 9f0dc829cbaa9aa5011fb917010d13ea5e0a19d7 (diff) | |
download | spark-d0216cb38bf976a3e98d1748e1a0ca76c8c7e713.tar.gz spark-d0216cb38bf976a3e98d1748e1a0ca76c8c7e713.tar.bz2 spark-d0216cb38bf976a3e98d1748e1a0ca76c8c7e713.zip |
Prevent DAGSchedulerSuite from corrupting driver.port.
Use the LocalSparkContext abstraction to properly manage clearing
spark.driver.port.
-rw-r--r-- | core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala | 9 |
1 files changed, 5 insertions, 4 deletions
diff --git a/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala index b3e6ab4c0f..3cdeb84674 100644 --- a/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala @@ -15,6 +15,8 @@ import org.easymock.{IAnswer, IArgumentMatcher} import akka.actor.ActorSystem +import spark.LocalSparkContext + import spark.storage.BlockManager import spark.storage.BlockManagerId import spark.storage.BlockManagerMaster @@ -42,13 +44,13 @@ import spark.{FetchFailed, Success} * DAGScheduler#submitWaitingStages (via test utility functions like runEvent or respondToTaskSet) * and capturing the resulting TaskSets from the mock TaskScheduler. */ -class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar with TimeLimitedTests { +class DAGSchedulerSuite extends FunSuite + with BeforeAndAfter with EasyMockSugar with TimeLimitedTests with LocalSparkContext { // impose a time limit on this test in case we don't let the job finish, in which case // JobWaiter#getResult will hang. override val timeLimit = Span(5, Seconds) - val sc: SparkContext = new SparkContext("local", "DAGSchedulerSuite") var scheduler: DAGScheduler = null val taskScheduler = mock[TaskScheduler] val blockManagerMaster = mock[BlockManagerMaster] @@ -105,6 +107,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar } before { + sc = new SparkContext("local", "DAGSchedulerSuite") taskSetMatchers.clear() cacheLocations.clear() val actorSystem = ActorSystem("test") @@ -125,8 +128,6 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar whenExecuting { scheduler.stop() } - sc.stop() - System.clearProperty("spark.master.port") } def makeBlockManagerId(host: String): BlockManagerId = |