-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-23040][CORE]: Returns interruptible iterator for shuffle reader #20449
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from 1 commit
acca0e3
ddeffd8
88e86e0
ba2f355
d6ed9a1
8c15c56
756e0b7
2061d0a
a3d8ad5
28119e9
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -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 = | ||
|
|
@@ -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. | ||
| 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() } | ||
|
||
| }) | ||
| 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) | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. there is a chance that
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Will do |
||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -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 | ||
|
|
@@ -320,6 +321,41 @@ class JobCancellationSuite extends SparkFunSuite with Matchers with BeforeAndAft | |
| f2.get() | ||
| } | ||
|
|
||
| test("Interruptible iterator of shuffle reader") { | ||
|
||
| 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() | ||
|
||
| // Small delay to ensure that foreach is cancelled if task is killed | ||
| Thread.sleep(1000) | ||
|
||
| iter | ||
| }.foreachAsync { _ => | ||
| executionOfInterruptibleCounter.getAndIncrement() | ||
|
||
| } | ||
|
|
||
| val sem = new Semaphore(0) | ||
| Future { | ||
| taskStartedSemaphore.acquire() | ||
| f.cancel() | ||
|
||
| sem.release() | ||
| } | ||
|
|
||
| sem.acquire() | ||
|
|
||
| val e = intercept[SparkException] { f.get() }.getCause | ||
|
||
|
|
||
| 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 | ||
| { | ||
|
|
@@ -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) | ||
| } | ||
There was a problem hiding this comment.
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)