-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-46170][SQL] Support inject adaptive query post planner strategy rules in SparkSessionExtensions #44074
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 |
|---|---|---|
|
|
@@ -47,6 +47,7 @@ import org.apache.spark.sql.execution.{ColumnarRule, SparkPlan} | |
| * <li>Customized Parser.</li> | ||
| * <li>(External) Catalog listeners.</li> | ||
| * <li>Columnar Rules.</li> | ||
| * <li>Adaptive Query Post Planner Strategy Rules.</li> | ||
| * <li>Adaptive Query Stage Preparation Rules.</li> | ||
| * <li>Adaptive Query Execution Runtime Optimizer Rules.</li> | ||
| * <li>Adaptive Query Stage Optimizer Rules.</li> | ||
|
|
@@ -114,12 +115,15 @@ class SparkSessionExtensions { | |
| type ColumnarRuleBuilder = SparkSession => ColumnarRule | ||
| type QueryStagePrepRuleBuilder = SparkSession => Rule[SparkPlan] | ||
| type QueryStageOptimizerRuleBuilder = SparkSession => Rule[SparkPlan] | ||
| type QueryPostPlannerStrategyBuilder = SparkSession => Rule[SparkPlan] | ||
|
|
||
| private[this] val columnarRuleBuilders = mutable.Buffer.empty[ColumnarRuleBuilder] | ||
| private[this] val queryStagePrepRuleBuilders = mutable.Buffer.empty[QueryStagePrepRuleBuilder] | ||
| private[this] val runtimeOptimizerRules = mutable.Buffer.empty[RuleBuilder] | ||
| private[this] val queryStageOptimizerRuleBuilders = | ||
| mutable.Buffer.empty[QueryStageOptimizerRuleBuilder] | ||
| private[this] val queryPostPlannerStrategyRuleBuilders = | ||
| mutable.Buffer.empty[QueryPostPlannerStrategyBuilder] | ||
|
||
|
|
||
| /** | ||
| * Build the override rules for columnar execution. | ||
|
|
@@ -149,6 +153,14 @@ class SparkSessionExtensions { | |
| queryStageOptimizerRuleBuilders.map(_.apply(session)).toSeq | ||
| } | ||
|
|
||
| /** | ||
| * Build the override rules for the query post planner strategy phase of adaptive query execution. | ||
| */ | ||
| private[sql] def buildQueryPostPlannerStrategyRules( | ||
| session: SparkSession): Seq[Rule[SparkPlan]] = { | ||
| queryPostPlannerStrategyRuleBuilders.map(_.apply(session)).toSeq | ||
| } | ||
|
|
||
|
Member
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. ditto. Shall we put this before |
||
| /** | ||
| * Inject a rule that can override the columnar execution of an executor. | ||
| */ | ||
|
|
@@ -185,6 +197,15 @@ class SparkSessionExtensions { | |
| queryStageOptimizerRuleBuilders += builder | ||
| } | ||
|
|
||
| /** | ||
| * Inject a rule that applied between `plannerStrategy` and `queryStagePrepRules`, so | ||
| * it can get the whole plan before injecting exchanges. | ||
| * Note, these rules can only be applied within AQE. | ||
| */ | ||
| def injectQueryPostPlannerStrategyRule(builder: QueryPostPlannerStrategyBuilder): Unit = { | ||
| queryPostPlannerStrategyRuleBuilders += builder | ||
| } | ||
|
|
||
|
||
| private[this] val resolutionRuleBuilders = mutable.Buffer.empty[RuleBuilder] | ||
|
|
||
| /** | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -29,15 +29,17 @@ import org.apache.spark.sql.catalyst.{FunctionIdentifier, InternalRow, TableIden | |
| import org.apache.spark.sql.catalyst.catalog.BucketSpec | ||
| import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec | ||
| import org.apache.spark.sql.catalyst.expressions._ | ||
| import org.apache.spark.sql.catalyst.expressions.aggregate.{Final, Partial} | ||
| import org.apache.spark.sql.catalyst.parser.{CatalystSqlParser, ParserInterface} | ||
| import org.apache.spark.sql.catalyst.plans.SQLHelper | ||
| import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, Limit, LocalRelation, LogicalPlan, Statistics, UnresolvedHint} | ||
| import org.apache.spark.sql.catalyst.plans.physical.Partitioning | ||
| import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, SinglePartition} | ||
| import org.apache.spark.sql.catalyst.rules.Rule | ||
| import org.apache.spark.sql.catalyst.trees.TreeNodeTag | ||
| import org.apache.spark.sql.connector.write.WriterCommitMessage | ||
| import org.apache.spark.sql.execution._ | ||
| import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanExec, AdaptiveSparkPlanHelper, AQEShuffleReadExec, QueryStageExec, ShuffleQueryStageExec} | ||
| import org.apache.spark.sql.execution.aggregate.HashAggregateExec | ||
| import org.apache.spark.sql.execution.datasources.{FileFormat, WriteFilesExec, WriteFilesSpec} | ||
| import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, BroadcastExchangeLike, ShuffleExchangeExec, ShuffleExchangeLike, ShuffleOrigin} | ||
| import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector | ||
|
|
@@ -516,6 +518,33 @@ class SparkSessionExtensionSuite extends SparkFunSuite with SQLHelper with Adapt | |
| } | ||
| } | ||
| } | ||
|
|
||
| test("SPARK-46170: Support inject adaptive query post planner strategy rules in " + | ||
| "SparkSessionExtensions") { | ||
| val extensions = create { extensions => | ||
| extensions.injectQueryPostPlannerStrategyRule(_ => MyQueryPostPlannerStrategyRule) | ||
| } | ||
| withSession(extensions) { session => | ||
| assert(session.sessionState.adaptiveRulesHolder.queryPostPlannerStrategyRules | ||
| .contains(MyQueryPostPlannerStrategyRule)) | ||
| import session.sqlContext.implicits._ | ||
|
||
| withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "3", | ||
| SQLConf.COALESCE_PARTITIONS_ENABLED.key -> "false") { | ||
| val input = Seq((10), (20), (10)).toDF("c1") | ||
|
||
| val df = input.groupBy("c1").count() | ||
| df.collect() | ||
|
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. I tried it, and even if
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. it is used to make sure we are checking the final plan |
||
| assert(df.rdd.partitions.length == 1) | ||
| assert(find(df.queryExecution.executedPlan) { | ||
| case s: ShuffleExchangeExec if s.outputPartitioning == SinglePartition => true | ||
|
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. How about Additionally, a personal opinion unrelated to this pr: If there is an |
||
| case _ => false | ||
| }.isDefined) | ||
| assert(find(df.queryExecution.executedPlan) { | ||
|
||
| case _: SortExec => true | ||
| case _ => false | ||
| }.isDefined) | ||
| } | ||
| } | ||
| } | ||
| } | ||
|
|
||
| case class MyRule(spark: SparkSession) extends Rule[LogicalPlan] { | ||
|
|
@@ -1190,3 +1219,14 @@ object RequireAtLeaseTwoPartitions extends Rule[SparkPlan] { | |
| } | ||
| } | ||
| } | ||
|
|
||
| object MyQueryPostPlannerStrategyRule extends Rule[SparkPlan] { | ||
| override def apply(plan: SparkPlan): SparkPlan = { | ||
| plan.transformUp { | ||
| case h: HashAggregateExec if h.aggregateExpressions.map(_.mode).contains(Partial) => | ||
| ShuffleExchangeExec(SinglePartition, h) | ||
| case h: HashAggregateExec if h.aggregateExpressions.map(_.mode).contains(Final) => | ||
| SortExec(h.groupingExpressions.map(k => SortOrder.apply(k, Ascending)), false, h) | ||
| } | ||
| } | ||
| } | ||
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.
Shall we put this before
type QueryStagePrepRuleBuilderbecause this rule is supposed to be used beforequeryStagePreparationRules?