-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-34637] [SQL] Support DPP + AQE when the broadcast exchange can be reused #31756
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 4 commits
3bc4baf
657c61b
c7b3735
6b07c84
4ccd4b8
701f1c3
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 |
|---|---|---|
|
|
@@ -17,19 +17,17 @@ | |
|
|
||
| package org.apache.spark.sql.execution.adaptive | ||
|
|
||
| import scala.collection.concurrent.TrieMap | ||
|
|
||
| import org.apache.spark.sql.catalyst.expressions.{BindReferences, DynamicPruningExpression, Literal} | ||
| import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight} | ||
| import org.apache.spark.sql.catalyst.rules.Rule | ||
| import org.apache.spark.sql.execution._ | ||
| import org.apache.spark.sql.execution.exchange.BroadcastExchangeExec | ||
| import org.apache.spark.sql.execution.joins.{HashedRelationBroadcastMode, HashJoin} | ||
| import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, HashedRelationBroadcastMode, HashJoin} | ||
|
|
||
| /** | ||
| * A rule to insert dynamic pruning predicates in order to reuse the results of broadcast. | ||
| */ | ||
| case class PlanAdaptiveDynamicPruningFilters( | ||
| stageCache: TrieMap[SparkPlan, QueryStageExec]) extends Rule[SparkPlan] { | ||
| case class PlanAdaptiveDynamicPruningFilters(rootPlan: SparkPlan) extends Rule[SparkPlan] { | ||
| def apply(plan: SparkPlan): SparkPlan = { | ||
| if (!conf.dynamicPartitionPruningEnabled) { | ||
| return plan | ||
|
|
@@ -44,12 +42,23 @@ case class PlanAdaptiveDynamicPruningFilters( | |
| val mode = HashedRelationBroadcastMode(packedKeys) | ||
| // plan a broadcast exchange of the build side of the join | ||
| val exchange = BroadcastExchangeExec(mode, adaptivePlan.executedPlan) | ||
| val existingStage = stageCache.get(exchange.canonicalized) | ||
| if (existingStage.nonEmpty && conf.exchangeReuseEnabled) { | ||
| val name = s"dynamicpruning#${exprId.id}" | ||
| val reuseQueryStage = existingStage.get.newReuseInstance(0, exchange.output) | ||
| val broadcastValues = | ||
| SubqueryBroadcastExec(name, index, buildKeys, reuseQueryStage) | ||
|
|
||
| val canReuseExchange = conf.exchangeReuseEnabled && buildKeys.nonEmpty && | ||
| rootPlan.find { | ||
| case BroadcastHashJoinExec(_, _, _, BuildLeft, _, left, _, _) => | ||
| left.sameResult(exchange) | ||
| case BroadcastHashJoinExec(_, _, _, BuildRight, _, _, right, _) => | ||
| right.sameResult(exchange) | ||
| case _ => false | ||
| }.isDefined | ||
|
|
||
| if (canReuseExchange) { | ||
| exchange.setLogicalLink(adaptivePlan.executedPlan.logicalLink.get) | ||
| val newAdaptivePlan = AdaptiveSparkPlanExec( | ||
cloud-fan marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| exchange, adaptivePlan.context, adaptivePlan.preprocessingRules, true) | ||
|
||
|
|
||
| val broadcastValues = SubqueryBroadcastExec( | ||
| name, index, buildKeys, newAdaptivePlan) | ||
| DynamicPruningExpression(InSubqueryExec(value, broadcastValues, exprId)) | ||
| } else { | ||
| DynamicPruningExpression(Literal.TrueLiteral) | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -197,6 +197,17 @@ abstract class DynamicPartitionPruningSuiteBase | |
| case _ => false | ||
| }.isDefined | ||
| assert(hasReuse, s"$s\nshould have been reused in\n$plan") | ||
| case a: AdaptiveSparkPlanExec => | ||
| val broadcastQueryStage = collectFirst(a) { | ||
| case b: BroadcastQueryStageExec => b | ||
| } | ||
| val broadcastPlan = broadcastQueryStage.get.broadcast | ||
| val hasReuse = find(plan) { | ||
| case ReusedExchangeExec(_, e) => e eq broadcastPlan | ||
| case b: BroadcastExchangeLike => b eq broadcastPlan | ||
| case _ => false | ||
| }.isDefined | ||
| assert(hasReuse, s"$s\nshould have been reused in\n$plan") | ||
| case _ => | ||
| fail(s"Invalid child node found in\n$s") | ||
| } | ||
|
|
@@ -1463,6 +1474,37 @@ abstract class DynamicPartitionPruningSuiteBase | |
| } | ||
| } | ||
| } | ||
|
|
||
| test("SPARK-34637: test DPP side broadcast query stage is created firstly") { | ||
cloud-fan marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true") { | ||
| val df = sql( | ||
| """ WITH view1 as ( | ||
|
||
| | SELECT f.store_id FROM fact_stats f WHERE f.units_sold = 70 group by f.store_id | ||
| | ) | ||
| | | ||
| | SELECT * FROM view1 v1 join view1 v2 WHERE v1.store_id = v2.store_id | ||
| """.stripMargin) | ||
|
|
||
| // A possible resulting query plan: | ||
| // BroadcastHashJoin | ||
| // +- HashAggregate | ||
| // +- ShuffleQueryStage | ||
| // +- Exchange | ||
| // +- HashAggregate | ||
| // +- Filter | ||
| // +- FileScan | ||
| // Dynamicpruning Subquery | ||
|
||
| // +- AdaptiveSparkPlan | ||
| // +- BroadcastQueryStage | ||
|
||
| // +- BroadcastExchange | ||
| // | ||
| // +- BroadcastQueryStage | ||
| // +- ReusedExchange | ||
|
|
||
| checkPartitionPruningPredicate(df, false, true) | ||
| checkAnswer(df, Row(15, 15) :: Nil) | ||
| } | ||
| } | ||
| } | ||
|
|
||
| class DynamicPartitionPruningSuiteAEOff extends DynamicPartitionPruningSuiteBase | ||
|
|
||

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.
We need use the
initialPlannot theinputPlan, because theinputPlanis not applied thequeryStagePreparationRules(EnsureRequirements).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 think it's better to pass
thisas the root plan.AdaptiveSparkPlanExeckeeps changing when more and more query stages are completed. So it's better thatPlanAdaptiveDynamicPruningFiltersalways look at the latest plan.