From dea4677c887a515e7b2a3ef52dd65e69b15c60c3 Mon Sep 17 00:00:00 2001
From: Kay Ousterhout <kayousterhout@gmail.com>
Date: Mon, 30 Sep 2013 16:44:45 -0700
Subject: [PATCH] Fixed compilation errors and broken test.

---
 .../cluster/ClusterTaskSetManager.scala          |  2 +-
 .../scheduler/local/LocalTaskSetManager.scala    |  4 ++--
 .../spark/scheduler/DAGSchedulerSuite.scala      |  2 --
 .../cluster/TaskResultGetterSuite.scala          | 16 ++++++++--------
 4 files changed, 11 insertions(+), 13 deletions(-)

diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManager.scala
index c7225de870..194ab55102 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManager.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManager.scala
@@ -28,7 +28,7 @@ import scala.math.min
 import scala.Some
 
 import org.apache.spark.{ExceptionFailure, FetchFailed, Logging, Resubmitted, SparkEnv,
-  SparkException, Success, TaskEndReason, TaskResultTooBigFailure, TaskState}
+  SparkException, Success, TaskEndReason, TaskResultLost, TaskState}
 import org.apache.spark.TaskState.TaskState
 import org.apache.spark.scheduler._
 import org.apache.spark.util.{SystemClock, Clock}
diff --git a/core/src/main/scala/org/apache/spark/scheduler/local/LocalTaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/local/LocalTaskSetManager.scala
index dc4cf555de..c2e2399ccb 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/local/LocalTaskSetManager.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/local/LocalTaskSetManager.scala
@@ -23,8 +23,8 @@ import scala.collection.mutable.HashMap
 
 import org.apache.spark.{ExceptionFailure, Logging, SparkEnv, SparkException, Success, TaskState}
 import org.apache.spark.TaskState.TaskState
-import org.apache.spark.scheduler.{Schedulable, Task, TaskDescription, TaskInfo, TaskLocality,
-  TaskResult, TaskSet, TaskSetManager}
+import org.apache.spark.scheduler.{DirectTaskResult, IndirectTaskResult, Pool, Schedulable, Task,
+  TaskDescription, TaskInfo, TaskLocality, TaskResult, TaskSet, TaskSetManager}
 
 
 private[spark] class LocalTaskSetManager(sched: LocalScheduler, val taskSet: TaskSet)
diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala
index 9ed591e494..2f933246b0 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala
@@ -32,8 +32,6 @@ import org.apache.spark.{Dependency, ShuffleDependency, OneToOneDependency}
 import org.apache.spark.{FetchFailed, Success, TaskEndReason}
 import org.apache.spark.storage.{BlockManagerId, BlockManagerMaster}
 
-import org.apache.spark.scheduler.Pool
-import org.apache.spark.scheduler.SchedulingMode
 import org.apache.spark.scheduler.SchedulingMode.SchedulingMode
 
 /**
diff --git a/core/src/test/scala/org/apache/spark/scheduler/cluster/TaskResultGetterSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/cluster/TaskResultGetterSuite.scala
index 2c201d15a7..119ba30090 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/cluster/TaskResultGetterSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/cluster/TaskResultGetterSuite.scala
@@ -19,8 +19,7 @@ package org.apache.spark.scheduler.cluster
 
 import java.nio.ByteBuffer
 
-import org.scalatest.BeforeAndAfter
-import org.scalatest.FunSuite
+import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll, FunSuite}
 
 import org.apache.spark.{LocalSparkContext, SparkContext, SparkEnv}
 import org.apache.spark.scheduler.{DirectTaskResult, IndirectTaskResult, TaskResult}
@@ -56,21 +55,22 @@ class ResultDeletingTaskResultGetter(sparkEnv: SparkEnv, scheduler: ClusterSched
 /**
  * Tests related to handling task results (both direct and indirect).
  */
-class TaskResultGetterSuite extends FunSuite with BeforeAndAfter with LocalSparkContext {
-
-  override def beforeAll() {
-    super.beforeAll()
+class TaskResultGetterSuite extends FunSuite with BeforeAndAfter with BeforeAndAfterAll
+  with LocalSparkContext {
 
+  override def beforeAll {
     // Set the Akka frame size to be as small as possible (it must be an integer, so 1 is as small
     // as we can make it) so the tests don't take too long.
     System.setProperty("spark.akka.frameSize", "1")
+  }
+
+  before {
     // Use local-cluster mode because results are returned differently when running with the
     // LocalScheduler.
     sc = new SparkContext("local-cluster[1,1,512]", "test")
   }
 
-  override def afterAll() {
-    super.afterAll()
+  override def afterAll {
     System.clearProperty("spark.akka.frameSize")
   }
 
-- 
GitLab