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 completed(either finished/cancelled).
Copy link
Contributor Author

Choose a reason for hiding this comment

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

To fit the 100 chars limitation, or is replaced by /

Copy link
Contributor

Choose a reason for hiding this comment

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

then we can just write 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

}
}
55 changes: 52 additions & 3 deletions core/src/test/scala/org/apache/spark/JobCancellationSuite.scala
Original file line number Diff line number Diff line change
Expand Up @@ -18,15 +18,14 @@
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
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 Down Expand Up @@ -320,6 +319,55 @@ class JobCancellationSuite extends SparkFunSuite with Matchers with BeforeAndAft
f2.get()
}

test("Interruptible iterator of shuffle reader") {
Copy link
Contributor

Choose a reason for hiding this comment

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

can we briefly explain what happened in this test?

import JobCancellationSuite._
val numSlice = 2
sc = new SparkContext(s"local[$numSlice]", "test")

val f = sc.parallelize(1 to 1000, numSlice).map { i => (i, i) }
.repartitionAndSortWithinPartitions(new HashPartitioner(2))
.mapPartitions { iter =>
taskStartedSemaphore.release()
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 be called twice as the root RDD has 2 partitions, so f.cancel might be called before both of these 2 partitions finished.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

f.cancel() should be called before these partitions(tasks) finishing , and we want to make sure these tasks could be cancelled

iter
}.foreachAsync { x =>
if ( x._1 >= 10) { // this block of code is partially executed.
Copy link
Contributor

Choose a reason for hiding this comment

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

no space after if(

taskCancelledSemaphore.acquire()
}
executionOfInterruptibleCounter.getAndIncrement()
Copy link
Contributor

Choose a reason for hiding this comment

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

do you mean without your PR, the task will keep running and hit this line 1000 times after canceling?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Yes

}

val sem = new Semaphore(0)
val taskCompletedSem = new Semaphore(0)
Future {
taskStartedSemaphore.acquire()
f.cancel()
Copy link
Contributor

Choose a reason for hiding this comment

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

what's the expectation for when this f.cancel() should be called?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Line 372: sem.acquire() is blocked by this Future block, but it looks we don't need Future or sem here. I will update the code.

sem.release()
}

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 task ends
taskCompletedSem.release()
}
}
})

sem.acquire()
val e = intercept[SparkException] { f.get() }.getCause
Copy link
Contributor

Choose a reason for hiding this comment

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

nit: intercept[SparkException](f.get()).getCause

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

assert(e.getMessage.contains("cancelled") || e.getMessage.contains("killed"))

taskCompletedSem.acquire(numSlice)
// 11 as |1..10| + |501|(another partition)
assert(executionOfInterruptibleCounter.get() <= 11)
Copy link
Contributor

Choose a reason for hiding this comment

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

why it's 11 not 10?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Copy link
Contributor

Choose a reason for hiding this comment

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

For simplicity, can we just test 1 partition/task?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Of course, we can. I choose 2 because Spark job normally has multiple partitions.

Then it's your call to go with single partition or stick with this one.

Copy link
Contributor

Choose a reason for hiding this comment

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

For a unit test, I'd like to have the simplest test that can expose the bug.

}

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