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,18 @@ 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 cancelled.
Copy link
Contributor

Choose a reason for hiding this comment

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

if task is completed(either finished or canceled)

context.addTaskCompletionListener(tc => {
// Note: we only stop sorter if cancelled as sorter.stop wouldn't be called in
// CompletionIterator. Another way would be making sorter.stop idempotent.
if (tc.isInterrupted()) { sorter.stop() }
Copy link
Contributor

Choose a reason for hiding this comment

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

seems we can remove this if if we don't return a CompletionIterator.

BTW I think we need to check all the places that use CompletionIterator, to see if they consider job 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.

One advantage of CompletionIterator is that the completionFunction will be called as soon as the wrapped iterator is consumed. So for sorter, it will release memory earlier rather than at task completion.

As for job cancelling, It's not just CompletionIterator that we should consider. The combiner and sorter pattern(or similar) is something we should look for:

combiner.insertAll(iterator) // or sorter.insertAll(iterator)
// then returns new iterator
combiner.iterator // or sorter.iterator

Copy link
Contributor

Choose a reason for hiding this comment

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

I may be missing something obvious, but seems ExternalSorter.stop() is already idempotent?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I may be missing something obvious, but seems ExternalSorter.stop() is already idempotent?

Ah, yes. After another look, it's indeed idempotent.
Will update the code.

})
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

}
}
37 changes: 37 additions & 0 deletions 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 Down Expand Up @@ -320,6 +321,41 @@ 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._
sc = new SparkContext("local[2]", "test")

val f = sc.parallelize(1 to 1000, 2).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

// Small delay to ensure that foreach is cancelled if task is killed
Thread.sleep(1000)
Copy link
Contributor

@jerryshao jerryshao Feb 8, 2018

Choose a reason for hiding this comment

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

I think using sleep will make the UT flaky, you'd better changing to some deterministic ways.

Copy link
Contributor

Choose a reason for hiding this comment

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

+1

iter
}.foreachAsync { _ =>
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)
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()
}

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(executionOfInterruptibleCounter.get() === 0)
assert(e.getMessage.contains("cancelled") || e.getMessage.contains("killed"))

// Small delay to ensure tasks are actually finished or killed
Thread.sleep(2000)
assert(executionOfInterruptibleCounter.get() === 0)

}

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