-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-49057][SQL] Do not block the AQE loop when submitting query stages #47533
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
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 |
|---|---|---|
|
|
@@ -61,23 +61,33 @@ trait BroadcastExchangeLike extends Exchange { | |
| */ | ||
| def relationFuture: Future[broadcast.Broadcast[Any]] | ||
|
|
||
| @transient | ||
| private lazy val triggerFuture: Future[Any] = { | ||
| SQLExecution.withThreadLocalCaptured(session, BroadcastExchangeExec.executionContext) { | ||
| // Trigger broadcast preparation which can involve expensive operations like waiting on | ||
| // subqueries and file listing. | ||
| executeQuery(null) | ||
ulysses-you marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| null | ||
| } | ||
| } | ||
|
|
||
| protected def completionFuture: scala.concurrent.Future[broadcast.Broadcast[Any]] | ||
|
|
||
| /** | ||
| * The asynchronous job that materializes the broadcast. It's used for registering callbacks on | ||
| * `relationFuture`. Note that calling this method may not start the execution of broadcast job. | ||
| * It also does the preparations work, such as waiting for the subqueries. | ||
| */ | ||
| final def submitBroadcastJob: scala.concurrent.Future[broadcast.Broadcast[Any]] = executeQuery { | ||
ulysses-you marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| materializationStarted.set(true) | ||
| final def submitBroadcastJob(): scala.concurrent.Future[broadcast.Broadcast[Any]] = { | ||
| triggerFuture | ||
| completionFuture | ||
| } | ||
|
|
||
| protected def completionFuture: scala.concurrent.Future[broadcast.Broadcast[Any]] | ||
|
|
||
| /** | ||
| * Cancels broadcast job with an optional reason. | ||
| */ | ||
| final def cancelBroadcastJob(reason: Option[String]): Unit = { | ||
| if (isMaterializationStarted() && !this.relationFuture.isDone) { | ||
| if (!this.relationFuture.isDone) { | ||
|
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. I do not re-implement broadcast cancellation, as we need more refactoring to move the creation of |
||
| reason match { | ||
| case Some(r) => sparkContext.cancelJobsWithTag(this.jobTag, r) | ||
| case None => sparkContext.cancelJobsWithTag(this.jobTag) | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -910,85 +910,49 @@ class AdaptiveQueryExecSuite | |
| } | ||
| } | ||
|
|
||
| test("SPARK-47148: AQE should avoid to materialize ShuffleQueryStage on the cancellation") { | ||
| test("SPARK-47148: AQE should avoid to submit shuffle job on cancellation") { | ||
|
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 test case seems to be a bit unstable:
any good solutions?
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. I think it's because the slow_udf is not slow enough and the shuffle stage was submitted too early. Can you try to increase the sleep time in slow_udf and see if it fixes the problem? |
||
| def createJoinedDF(): DataFrame = { | ||
| val df = spark.range(5).toDF("col") | ||
| val df2 = spark.range(10).toDF("col").coalesce(2) | ||
| val df3 = spark.range(15).toDF("col").filter(Symbol("col") >= 2) | ||
| df.join(df2, Seq("col")).join(df3, Seq("col")) | ||
| // Use subquery expression containing `slow_udf` to delay the submission of shuffle jobs. | ||
| val df = sql("SELECT id, (SELECT slow_udf() FROM range(2)) FROM range(5)") | ||
| val df2 = sql("SELECT id FROM range(10)").coalesce(2) | ||
| val df3 = sql("SELECT id, (SELECT slow_udf() FROM range(2)) FROM range(15) WHERE id > 2") | ||
| df.join(df2, Seq("id")).join(df3, Seq("id")) | ||
| } | ||
|
|
||
| try { | ||
| spark.experimental.extraStrategies = TestProblematicCoalesceStrategy :: Nil | ||
| withSQLConf( | ||
| SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", | ||
| SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { | ||
| val joinedDF = createJoinedDF() | ||
|
|
||
| val error = intercept[SparkException] { | ||
| joinedDF.collect() | ||
| } | ||
| assert(error.getMessage() contains "ProblematicCoalesce execution is failed") | ||
|
|
||
| val adaptivePlan = joinedDF.queryExecution.executedPlan.asInstanceOf[AdaptiveSparkPlanExec] | ||
|
|
||
| // All QueryStages should be based on ShuffleQueryStageExec | ||
| val shuffleQueryStageExecs = collect(adaptivePlan) { | ||
| case sqse: ShuffleQueryStageExec => sqse | ||
| } | ||
| assert(shuffleQueryStageExecs.length == 3, s"Physical Plan should include " + | ||
| s"3 ShuffleQueryStages. Physical Plan: $adaptivePlan") | ||
| shuffleQueryStageExecs.foreach(sqse => assert(sqse.name.contains("ShuffleQueryStageExec-"))) | ||
| // First ShuffleQueryStage is materialized so it needs to be canceled. | ||
| assert(shuffleQueryStageExecs(0).shuffle.isMaterializationStarted(), | ||
| "Materialization should be started.") | ||
| // Second ShuffleQueryStage materialization is failed so | ||
| // it is excluded from the cancellation due to earlyFailedStage. | ||
| assert(shuffleQueryStageExecs(1).shuffle.isMaterializationStarted(), | ||
| "Materialization should be started but it is failed.") | ||
| // Last ShuffleQueryStage is not materialized yet so it does not require | ||
| // to be canceled and it is just skipped from the cancellation. | ||
| assert(!shuffleQueryStageExecs(2).shuffle.isMaterializationStarted(), | ||
| "Materialization should not be started.") | ||
| } | ||
| } finally { | ||
| spark.experimental.extraStrategies = Nil | ||
| } | ||
| } | ||
| withUserDefinedFunction("slow_udf" -> true) { | ||
| spark.udf.register("slow_udf", () => { | ||
| Thread.sleep(3000) | ||
| 1 | ||
| }) | ||
|
|
||
| test("SPARK-47148: Check if BroadcastQueryStage materialization is started") { | ||
| def createJoinedDF(): DataFrame = { | ||
| spark.range(10).toDF("col1").createTempView("t1") | ||
| spark.range(5).coalesce(2).toDF("col2").createTempView("t2") | ||
| spark.range(15).toDF("col3").filter(Symbol("col3") >= 2).createTempView("t3") | ||
| sql("SELECT /*+ BROADCAST(t3) */ * FROM (SELECT /*+ BROADCAST(t2) */ * FROM t1 " + | ||
| "INNER JOIN t2 ON t1.col1 = t2.col2) t JOIN t3 ON t.col1 = t3.col3;") | ||
| } | ||
| withTempView("t1", "t2", "t3") { | ||
| try { | ||
| spark.experimental.extraStrategies = TestProblematicCoalesceStrategy :: Nil | ||
| withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true") { | ||
| val joinedDF = createJoinedDF() | ||
| withSQLConf( | ||
| SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", | ||
| SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { | ||
| val joined = createJoinedDF() | ||
| joined.explain(true) | ||
|
|
||
| val error = intercept[SparkException] { | ||
| joinedDF.collect() | ||
| joined.collect() | ||
| } | ||
| assert(error.getMessage() contains "ProblematicCoalesce execution is failed") | ||
| assert(error.getMessage() contains "coalesce test error") | ||
|
|
||
| val adaptivePlan = | ||
| joinedDF.queryExecution.executedPlan.asInstanceOf[AdaptiveSparkPlanExec] | ||
| val adaptivePlan = joined.queryExecution.executedPlan.asInstanceOf[AdaptiveSparkPlanExec] | ||
|
|
||
| // All QueryStages should be based on BroadcastQueryStageExec | ||
| val broadcastQueryStageExecs = collect(adaptivePlan) { | ||
| case bqse: BroadcastQueryStageExec => bqse | ||
| } | ||
| assert(broadcastQueryStageExecs.length == 2, adaptivePlan) | ||
| broadcastQueryStageExecs.foreach { bqse => | ||
| assert(bqse.name.contains("BroadcastQueryStageExec-")) | ||
| // Both BroadcastQueryStages are materialized at the beginning. | ||
| assert(bqse.broadcast.isMaterializationStarted(), | ||
| s"${bqse.name}' s materialization should be started.") | ||
| // All QueryStages should be based on ShuffleQueryStageExec | ||
| val shuffleQueryStageExecs = collect(adaptivePlan) { | ||
| case sqse: ShuffleQueryStageExec => sqse | ||
| } | ||
| assert(shuffleQueryStageExecs.length == 3, s"Physical Plan should include " + | ||
| s"3 ShuffleQueryStages. Physical Plan: $adaptivePlan") | ||
| // First ShuffleQueryStage is cancelled before shuffle job is submitted. | ||
| assert(shuffleQueryStageExecs(0).shuffle.futureAction.get.isEmpty) | ||
| // Second ShuffleQueryStage has submitted the shuffle job but it failed. | ||
| assert(shuffleQueryStageExecs(1).shuffle.futureAction.get.isDefined, | ||
| "Materialization should be started but it is failed.") | ||
| // Third ShuffleQueryStage is cancelled before shuffle job is submitted. | ||
| assert(shuffleQueryStageExecs(2).shuffle.futureAction.get.isEmpty) | ||
| } | ||
| } finally { | ||
| spark.experimental.extraStrategies = Nil | ||
|
|
@@ -3057,8 +3021,11 @@ private case class SimpleShuffleSortCostEvaluator() extends CostEvaluator { | |
| private object TestProblematicCoalesceStrategy extends Strategy { | ||
| private case class TestProblematicCoalesceExec(numPartitions: Int, child: SparkPlan) | ||
| extends UnaryExecNode { | ||
| override protected def doExecute(): RDD[InternalRow] = | ||
| throw new SparkException("ProblematicCoalesce execution is failed") | ||
| override protected def doExecute(): RDD[InternalRow] = { | ||
| child.execute().mapPartitions { _ => | ||
| throw new RuntimeException("coalesce test error") | ||
| } | ||
| } | ||
| override def output: Seq[Attribute] = child.output | ||
| override protected def withNewChildInternal(newChild: SparkPlan): TestProblematicCoalesceExec = | ||
| copy(child = newChild) | ||
|
|
||
Uh oh!
There was an error while loading. Please reload this page.
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.
Can you explain why we pick this number? It might create memory pressure on the driver
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.
The shuffle async job is just waiting for other work (subquery expression execution) to finish, which is very light-weighted. The broadcast async job executes a query and collects the result in the driver, which is very heavy. That's why we can give much larger parallelism to the shuffle async jobs. In our benchmark we found this number is reasonably good for TPC.
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.
Is there a correlation with the number of system cores?
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.
I don't think so, the
BROADCAST_EXCHANGE_MAX_THREAD_THRESHOLDis also way larger than the driver system cores.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.
I'm not sure if this parameter has anything to do with SPARK-49091 or if it was just caused by SPARK-41914 which the reporter pointed to.
Also cc @wangyum
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.
Update: SPARK-49091 is not related