-
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 9 commits
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 |
|---|---|---|
|
|
@@ -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 | ||
|
|
@@ -26,7 +27,7 @@ import scala.concurrent.duration._ | |
| import org.scalatest.BeforeAndAfter | ||
| import org.scalatest.Matchers | ||
|
|
||
|
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. 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 | ||
|
|
||
| /** | ||
|
|
@@ -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() | ||
|
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. nit: for simplicity, I'd like to reset all semaphores here, instead of thinking about which one are shared.
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. or we can make all semaphores local, so that we don't need to care about it.
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.
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
No, Global semaphore is required when being shared between driver and executor(another thread in local mode).
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.
+1. It's also good for reviewers, otherwise figuring out a semaphore is shared or not is really unnecessary for reviewers. |
||
| } finally { | ||
| super.afterEach() | ||
| } | ||
|
|
@@ -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() | ||
|
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. why not
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. 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() | ||
|
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. We should add some comment to explain when we reach here. From what I am seeing:
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 |
||
|
|
||
| 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 | ||
| { | ||
|
|
@@ -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) | ||
| } | ||
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.
there is a chance that
resultIteris already anInterruptibleIterator, 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.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.
Will do