Skip to content
Original file line number Diff line number Diff line change
Expand Up @@ -95,7 +95,7 @@ private[spark] class BlockStoreShuffleReader[K, C](
}

// Sort the output if there is a sort ordering defined.
dep.keyOrdering match {
val resultIter = dep.keyOrdering match {
case Some(keyOrd: Ordering[K]) =>
// Create an ExternalSorter to sort the data.
val sorter =
Expand All @@ -104,9 +104,16 @@ private[spark] class BlockStoreShuffleReader[K, C](
context.taskMetrics().incMemoryBytesSpilled(sorter.memoryBytesSpilled)
context.taskMetrics().incDiskBytesSpilled(sorter.diskBytesSpilled)
context.taskMetrics().incPeakExecutionMemory(sorter.peakMemoryUsedBytes)
// Use completion callback to stop sorter if task was finished/cancelled.
context.addTaskCompletionListener(_ => {
sorter.stop()
})
CompletionIterator[Product2[K, C], Iterator[Product2[K, C]]](sorter.iterator, sorter.stop())
case None =>
aggregatedIter
}
// Use another interruptible iterator here to support task cancellation as aggregator or(and)
// sorter may have consumed previous interruptible iterator.
new InterruptibleIterator[Product2[K, C]](context, resultIter)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

there is a chance that resultIter is already an InterruptibleIterator, and we should not double wrap it. Can you send a followup PR to fix this? then we can backport them to 2.3 together.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Will do

}
}
64 changes: 63 additions & 1 deletion core/src/test/scala/org/apache/spark/JobCancellationSuite.scala
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@
package org.apache.spark

import java.util.concurrent.Semaphore
import java.util.concurrent.atomic.AtomicInteger

import scala.concurrent.ExecutionContext.Implicits.global
import scala.concurrent.Future
Expand All @@ -26,7 +27,7 @@ import scala.concurrent.duration._
import org.scalatest.BeforeAndAfter
import org.scalatest.Matchers

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

this will break the style check

import org.apache.spark.scheduler.{SparkListener, SparkListenerTaskStart}
import org.apache.spark.scheduler.{SparkListener, SparkListenerStageCompleted, SparkListenerTaskEnd, SparkListenerTaskStart}
import org.apache.spark.util.ThreadUtils

/**
Expand All @@ -40,6 +41,10 @@ class JobCancellationSuite extends SparkFunSuite with Matchers with BeforeAndAft
override def afterEach() {
try {
resetSparkContext()
// Reset semaphores if used by multiple tests.
// Note: if other semaphores are shared by multiple tests, please reset them in this block
JobCancellationSuite.taskStartedSemaphore.drainPermits()
JobCancellationSuite.taskCancelledSemaphore.drainPermits()
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: for simplicity, I'd like to reset all semaphores here, instead of thinking about which one are shared.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

or we can make all semaphores local, so that we don't need to care about it.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

for simplicity, I'd like to reset all semaphores here, instead of thinking about which one are shared.

Another way to avoid this problem is: don't reuse semaphores. But that's too verbose.

As for your suggestion, if new semaphores are added by others, how could he know that he's supposed to reset the semaphores? Maybe some comments are needed in semaphore declaration

or we can make all semaphores local, so that we don't need to care about it.

No, Global semaphore is required when being shared between driver and executor(another thread in local mode).
See related pr #4180 for details

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Maybe some comments are needed in semaphore declaration

+1. It's also good for reviewers, otherwise figuring out a semaphore is shared or not is really unnecessary for reviewers.

} finally {
super.afterEach()
}
Expand Down Expand Up @@ -320,6 +325,62 @@ class JobCancellationSuite extends SparkFunSuite with Matchers with BeforeAndAft
f2.get()
}

test("interruptible iterator of shuffle reader") {
// In this test case, we create a Spark job of two stages. The second stage is cancelled during
// execution and a counter is used to make sure that the corresponding tasks are indeed
// cancelled.
import JobCancellationSuite._
sc = new SparkContext("local[2]", "test interruptible iterator")

val taskCompletedSem = new Semaphore(0)

sc.addSparkListener(new SparkListener {
override def onStageCompleted(stageCompleted: SparkListenerStageCompleted): Unit = {
// release taskCancelledSemaphore when cancelTasks event has been posted
if (stageCompleted.stageInfo.stageId == 1) {
taskCancelledSemaphore.release(1000)
}
}

override def onTaskEnd(taskEnd: SparkListenerTaskEnd): Unit = {
if (taskEnd.stageId == 1) { // make sure tasks are completed
taskCompletedSem.release()
}
}
})

val f = sc.parallelize(1 to 1000).map { i => (i, i) }
.repartitionAndSortWithinPartitions(new HashPartitioner(1))
.mapPartitions { iter =>
taskStartedSemaphore.release()
iter
}.foreachAsync { x =>
if (x._1 >= 10) {
// This block of code is partially executed. It will be blocked when x._1 >= 10 and the
// next iteration will be cancelled if the source iterator is interruptible. Then in this
// case, the maximum num of increment would be 10(|1...10|)
taskCancelledSemaphore.acquire()
}
executionOfInterruptibleCounter.getAndIncrement()
}

taskStartedSemaphore.acquire()
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

why not taskStartedSemaphore.acquire(numSlice)?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

As soon as one task starts, we can cancel the job.

// Job is cancelled when:
// 1. task in reduce stage has been started, guaranteed by previous line.
// 2. task in reduce stage is blocked after processing at most 10 records as
// taskCancelledSemaphore is not released until cancelTasks event is posted
// After job being cancelled, task in reduce stage will be cancelled and no more iteration are
// executed.
f.cancel()
Copy link
Contributor

@cloud-fan cloud-fan Feb 27, 2018

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We should add some comment to explain when we reach here. From what I am seeing:

  1. taskStartedSemaphore.release() must be called, so at least one task is started.
  2. the first task has processed no more than 10 records, the second task hasn't processed any data, because the reduce stage is not finished and taskCancelledSemaphore.acquire() will be blocked.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

will do


val e = intercept[SparkException](f.get()).getCause
assert(e.getMessage.contains("cancelled") || e.getMessage.contains("killed"))

// Make sure tasks are indeed completed.
taskCompletedSem.acquire()
assert(executionOfInterruptibleCounter.get() <= 10)
}

def testCount() {
// Cancel before launching any tasks
{
Expand Down Expand Up @@ -384,4 +445,5 @@ object JobCancellationSuite {
val taskStartedSemaphore = new Semaphore(0)
val taskCancelledSemaphore = new Semaphore(0)
val twoJobsSharingStageSemaphore = new Semaphore(0)
val executionOfInterruptibleCounter = new AtomicInteger(0)
}