aboutsummaryrefslogtreecommitdiff
diff options
context:
space:
mode:
authorMatei Zaharia <matei@eecs.berkeley.edu>2013-03-10 16:52:54 -0700
committerMatei Zaharia <matei@eecs.berkeley.edu>2013-03-10 16:52:54 -0700
commit2e1bbc4e7ec13717f235e5ba9c136dd711eb603c (patch)
tree359877de433101d35d5ef7bd3f42078cdae0e6d1
parentd4e29ea878b6ac31fa5758a3e262f87d6e9b928d (diff)
parentd0216cb38bf976a3e98d1748e1a0ca76c8c7e713 (diff)
downloadspark-2e1bbc4e7ec13717f235e5ba9c136dd711eb603c.tar.gz
spark-2e1bbc4e7ec13717f235e5ba9c136dd711eb603c.tar.bz2
spark-2e1bbc4e7ec13717f235e5ba9c136dd711eb603c.zip
Merge remote-tracking branch 'woggling/dag-sched-driver-port'
Conflicts: core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala
-rw-r--r--core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala9
1 files changed, 4 insertions, 5 deletions
diff --git a/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala
index 710df929f6..6da58a0f6e 100644
--- a/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala
+++ b/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala
@@ -5,6 +5,8 @@ import scala.collection.mutable.{Map, HashMap}
import org.scalatest.FunSuite
import org.scalatest.BeforeAndAfter
+import spark.LocalSparkContext
+
import spark.storage.BlockManager
import spark.storage.BlockManagerId
import spark.storage.BlockManagerMaster
@@ -32,9 +34,7 @@ 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 {
-
- val sc: SparkContext = new SparkContext("local", "DAGSchedulerSuite")
+class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkContext {
/** Set of TaskSets the DAGScheduler has requested executed. */
val taskSets = scala.collection.mutable.Buffer[TaskSet]()
@@ -86,6 +86,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter {
}
before {
+ sc = new SparkContext("local", "DAGSchedulerSuite")
taskSets.clear()
cacheLocations.clear()
results.clear()
@@ -100,8 +101,6 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter {
after {
scheduler.stop()
- sc.stop()
- System.clearProperty("spark.master.port")
}
/**