diff --git a/core/src/main/scala/org/apache/spark/TaskContext.scala b/core/src/main/scala/org/apache/spark/TaskContext.scala
index fc4812753d005b0087203f32090fc0b913844d4b..51f40c339d13cd6407fe43098e4eb5e89567c7c4 100644
--- a/core/src/main/scala/org/apache/spark/TaskContext.scala
+++ b/core/src/main/scala/org/apache/spark/TaskContext.scala
@@ -28,13 +28,12 @@ import org.apache.spark.executor.TaskMetrics
  */
 @DeveloperApi
 class TaskContext(
-  val stageId: Int,
-  val partitionId: Int,
-  val attemptId: Long,
-  val runningLocally: Boolean = false,
-  @volatile var interrupted: Boolean = false,
-  private[spark] val taskMetrics: TaskMetrics = TaskMetrics.empty
-) extends Serializable {
+    val stageId: Int,
+    val partitionId: Int,
+    val attemptId: Long,
+    val runningLocally: Boolean = false,
+    private[spark] val taskMetrics: TaskMetrics = TaskMetrics.empty)
+  extends Serializable {
 
   @deprecated("use partitionId", "0.8.1")
   def splitId = partitionId
@@ -42,7 +41,10 @@ class TaskContext(
   // List of callback functions to execute when the task completes.
   @transient private val onCompleteCallbacks = new ArrayBuffer[() => Unit]
 
-  // Set to true when the task is completed, before the onCompleteCallbacks are executed.
+  // Whether the corresponding task has been killed.
+  @volatile var interrupted: Boolean = false
+
+  // Whether the task has completed, before the onCompleteCallbacks are executed.
   @volatile var completed: Boolean = false
 
   /**
@@ -58,6 +60,6 @@ class TaskContext(
   def executeOnCompleteCallbacks() {
     completed = true
     // Process complete callbacks in the reverse order of registration
-    onCompleteCallbacks.reverse.foreach{_()}
+    onCompleteCallbacks.reverse.foreach { _() }
   }
 }
diff --git a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala
index 2259df0b56badeaf128f8ef79e7fa7ca2dbcda41..4b0324f2b5447d52be7c038cc6c3a83845d677d1 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala
@@ -23,7 +23,6 @@ import java.io._
 import java.util.zip.{GZIPInputStream, GZIPOutputStream}
 
 import scala.collection.mutable.HashMap
-import scala.util.Try
 
 import org.apache.spark._
 import org.apache.spark.executor.ShuffleWriteMetrics
@@ -70,7 +69,7 @@ private[spark] object ShuffleMapTask {
   }
 
   // Since both the JarSet and FileSet have the same format this is used for both.
-  def deserializeFileSet(bytes: Array[Byte]) : HashMap[String, Long] = {
+  def deserializeFileSet(bytes: Array[Byte]): HashMap[String, Long] = {
     val in = new GZIPInputStream(new ByteArrayInputStream(bytes))
     val objIn = new ObjectInputStream(in)
     val set = objIn.readObject().asInstanceOf[Array[(String, Long)]].toMap
diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java
index c3e03cea917b3901aa92377d540ac0e57559da5d..1912015827927e214c95b7df3bb8c0b8468d89f0 100644
--- a/core/src/test/java/org/apache/spark/JavaAPISuite.java
+++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java
@@ -597,7 +597,7 @@ public class JavaAPISuite implements Serializable {
   @Test
   public void iterator() {
     JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5), 2);
-    TaskContext context = new TaskContext(0, 0, 0, false, false, new TaskMetrics());
+    TaskContext context = new TaskContext(0, 0, 0, false, new TaskMetrics());
     Assert.assertEquals(1, rdd.iterator(rdd.splits().get(0), context).next().intValue());
   }
 
diff --git a/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala b/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala
index fd5b0906e67650e222833007d2edd881ff546dc9..4f178db40f638af35c7e1484616a87f6e204d8bc 100644
--- a/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala
@@ -23,7 +23,6 @@ import org.scalatest.{BeforeAndAfter, FunSuite}
 import org.scalatest.mock.EasyMockSugar
 
 import org.apache.spark.rdd.RDD
-import org.apache.spark.executor.TaskMetrics
 import org.apache.spark.storage._
 
 // TODO: Test the CacheManager's thread-safety aspects
@@ -59,8 +58,7 @@ class CacheManagerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar
     }
 
     whenExecuting(blockManager) {
-      val context = new TaskContext(0, 0, 0, interrupted = false, runningLocally = false,
-        taskMetrics = TaskMetrics.empty)
+      val context = new TaskContext(0, 0, 0)
       val value = cacheManager.getOrCompute(rdd, split, context, StorageLevel.MEMORY_ONLY)
       assert(value.toList === List(1, 2, 3, 4))
     }
@@ -72,8 +70,7 @@ class CacheManagerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar
     }
 
     whenExecuting(blockManager) {
-      val context = new TaskContext(0, 0, 0, interrupted = false, runningLocally = false,
-        taskMetrics = TaskMetrics.empty)
+      val context = new TaskContext(0, 0, 0)
       val value = cacheManager.getOrCompute(rdd, split, context, StorageLevel.MEMORY_ONLY)
       assert(value.toList === List(5, 6, 7))
     }
@@ -86,8 +83,7 @@ class CacheManagerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar
     }
 
     whenExecuting(blockManager) {
-      val context = new TaskContext(0, 0, 0, runningLocally = true, interrupted = false,
-        taskMetrics = TaskMetrics.empty)
+      val context = new TaskContext(0, 0, 0, runningLocally = true)
       val value = cacheManager.getOrCompute(rdd, split, context, StorageLevel.MEMORY_ONLY)
       assert(value.toList === List(1, 2, 3, 4))
     }
diff --git a/core/src/test/scala/org/apache/spark/PipedRDDSuite.scala b/core/src/test/scala/org/apache/spark/PipedRDDSuite.scala
index 0bb6a6b09c5b5872deb40efb5ec6258c79b69350..db56a4acdd6f57d8dcebfdf167bd8225960b88db 100644
--- a/core/src/test/scala/org/apache/spark/PipedRDDSuite.scala
+++ b/core/src/test/scala/org/apache/spark/PipedRDDSuite.scala
@@ -178,14 +178,12 @@ class PipedRDDSuite extends FunSuite with SharedSparkContext {
       }
       val hadoopPart1 = generateFakeHadoopPartition()
       val pipedRdd = new PipedRDD(nums, "printenv " + varName)
-      val tContext = new TaskContext(0, 0, 0, interrupted = false, runningLocally = false,
-        taskMetrics = TaskMetrics.empty)
+      val tContext = new TaskContext(0, 0, 0)
       val rddIter = pipedRdd.compute(hadoopPart1, tContext)
       val arr = rddIter.toArray
       assert(arr(0) == "/some/path")
     } else {
       // printenv isn't available so just pass the test
-      assert(true)
     }
   }