Skip to content

Commit c3f8b78

Browse files
andrewor14aarondav
authored andcommitted
[SPARK-1745] Move interrupted flag from TaskContext constructor (minor)
It makes little sense to start a TaskContext that is interrupted. Indeed, I searched for all use cases of it and didn't find a single instance in which `interrupted` is true on construction. This was inspired by reviewing #640, which adds an additional `@volatile var completed` that is similar. These are not the most urgent changes, but I wanted to push them out before I forget. Author: Andrew Or <[email protected]> Closes #675 from andrewor14/task-context and squashes the following commits: 9575e02 [Andrew Or] Add space 69455d1 [Andrew Or] Merge branch 'master' of github.com:apache/spark into task-context c471490 [Andrew Or] Oops, removed one flag too many. Adding it back. 85311f8 [Andrew Or] Move interrupted flag from TaskContext constructor
1 parent 44dd57f commit c3f8b78

File tree

5 files changed

+17
-22
lines changed

5 files changed

+17
-22
lines changed

core/src/main/scala/org/apache/spark/TaskContext.scala

Lines changed: 11 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -28,21 +28,23 @@ import org.apache.spark.executor.TaskMetrics
2828
*/
2929
@DeveloperApi
3030
class TaskContext(
31-
val stageId: Int,
32-
val partitionId: Int,
33-
val attemptId: Long,
34-
val runningLocally: Boolean = false,
35-
@volatile var interrupted: Boolean = false,
36-
private[spark] val taskMetrics: TaskMetrics = TaskMetrics.empty
37-
) extends Serializable {
31+
val stageId: Int,
32+
val partitionId: Int,
33+
val attemptId: Long,
34+
val runningLocally: Boolean = false,
35+
private[spark] val taskMetrics: TaskMetrics = TaskMetrics.empty)
36+
extends Serializable {
3837

3938
@deprecated("use partitionId", "0.8.1")
4039
def splitId = partitionId
4140

4241
// List of callback functions to execute when the task completes.
4342
@transient private val onCompleteCallbacks = new ArrayBuffer[() => Unit]
4443

45-
// Set to true when the task is completed, before the onCompleteCallbacks are executed.
44+
// Whether the corresponding task has been killed.
45+
@volatile var interrupted: Boolean = false
46+
47+
// Whether the task has completed, before the onCompleteCallbacks are executed.
4648
@volatile var completed: Boolean = false
4749

4850
/**
@@ -58,6 +60,6 @@ class TaskContext(
5860
def executeOnCompleteCallbacks() {
5961
completed = true
6062
// Process complete callbacks in the reverse order of registration
61-
onCompleteCallbacks.reverse.foreach{_()}
63+
onCompleteCallbacks.reverse.foreach { _() }
6264
}
6365
}

core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala

Lines changed: 1 addition & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -23,7 +23,6 @@ import java.io._
2323
import java.util.zip.{GZIPInputStream, GZIPOutputStream}
2424

2525
import scala.collection.mutable.HashMap
26-
import scala.util.Try
2726

2827
import org.apache.spark._
2928
import org.apache.spark.executor.ShuffleWriteMetrics
@@ -70,7 +69,7 @@ private[spark] object ShuffleMapTask {
7069
}
7170

7271
// Since both the JarSet and FileSet have the same format this is used for both.
73-
def deserializeFileSet(bytes: Array[Byte]) : HashMap[String, Long] = {
72+
def deserializeFileSet(bytes: Array[Byte]): HashMap[String, Long] = {
7473
val in = new GZIPInputStream(new ByteArrayInputStream(bytes))
7574
val objIn = new ObjectInputStream(in)
7675
val set = objIn.readObject().asInstanceOf[Array[(String, Long)]].toMap

core/src/test/java/org/apache/spark/JavaAPISuite.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -597,7 +597,7 @@ public void persist() {
597597
@Test
598598
public void iterator() {
599599
JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5), 2);
600-
TaskContext context = new TaskContext(0, 0, 0, false, false, new TaskMetrics());
600+
TaskContext context = new TaskContext(0, 0, 0, false, new TaskMetrics());
601601
Assert.assertEquals(1, rdd.iterator(rdd.splits().get(0), context).next().intValue());
602602
}
603603

core/src/test/scala/org/apache/spark/CacheManagerSuite.scala

Lines changed: 3 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -23,7 +23,6 @@ import org.scalatest.{BeforeAndAfter, FunSuite}
2323
import org.scalatest.mock.EasyMockSugar
2424

2525
import org.apache.spark.rdd.RDD
26-
import org.apache.spark.executor.TaskMetrics
2726
import org.apache.spark.storage._
2827

2928
// TODO: Test the CacheManager's thread-safety aspects
@@ -59,8 +58,7 @@ class CacheManagerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar
5958
}
6059

6160
whenExecuting(blockManager) {
62-
val context = new TaskContext(0, 0, 0, interrupted = false, runningLocally = false,
63-
taskMetrics = TaskMetrics.empty)
61+
val context = new TaskContext(0, 0, 0)
6462
val value = cacheManager.getOrCompute(rdd, split, context, StorageLevel.MEMORY_ONLY)
6563
assert(value.toList === List(1, 2, 3, 4))
6664
}
@@ -72,8 +70,7 @@ class CacheManagerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar
7270
}
7371

7472
whenExecuting(blockManager) {
75-
val context = new TaskContext(0, 0, 0, interrupted = false, runningLocally = false,
76-
taskMetrics = TaskMetrics.empty)
73+
val context = new TaskContext(0, 0, 0)
7774
val value = cacheManager.getOrCompute(rdd, split, context, StorageLevel.MEMORY_ONLY)
7875
assert(value.toList === List(5, 6, 7))
7976
}
@@ -86,8 +83,7 @@ class CacheManagerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar
8683
}
8784

8885
whenExecuting(blockManager) {
89-
val context = new TaskContext(0, 0, 0, runningLocally = true, interrupted = false,
90-
taskMetrics = TaskMetrics.empty)
86+
val context = new TaskContext(0, 0, 0, runningLocally = true)
9187
val value = cacheManager.getOrCompute(rdd, split, context, StorageLevel.MEMORY_ONLY)
9288
assert(value.toList === List(1, 2, 3, 4))
9389
}

core/src/test/scala/org/apache/spark/PipedRDDSuite.scala

Lines changed: 1 addition & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -178,14 +178,12 @@ class PipedRDDSuite extends FunSuite with SharedSparkContext {
178178
}
179179
val hadoopPart1 = generateFakeHadoopPartition()
180180
val pipedRdd = new PipedRDD(nums, "printenv " + varName)
181-
val tContext = new TaskContext(0, 0, 0, interrupted = false, runningLocally = false,
182-
taskMetrics = TaskMetrics.empty)
181+
val tContext = new TaskContext(0, 0, 0)
183182
val rddIter = pipedRdd.compute(hadoopPart1, tContext)
184183
val arr = rddIter.toArray
185184
assert(arr(0) == "/some/path")
186185
} else {
187186
// printenv isn't available so just pass the test
188-
assert(true)
189187
}
190188
}
191189

0 commit comments

Comments
 (0)