diff --git a/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala
index 710df929f6afe8802357a5437176df5d485c1103..6da58a0f6e5fe3beeb6117382960e3c57ab679e9 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")
   }
 
   /**