-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-41048][SQL] Improve output partitioning and ordering with AQE cache #38558
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
Closed
Closed
Changes from 1 commit
Commits
Show all changes
3 commits
Select commit
Hold shift + click to select a range
File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -40,10 +40,10 @@ import org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation | |
| import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, LeafNode, LocalRelation, LogicalPlan, OneRowRelation, Statistics} | ||
| import org.apache.spark.sql.catalyst.util.DateTimeUtils | ||
| import org.apache.spark.sql.connector.FakeV2Provider | ||
| import org.apache.spark.sql.execution.{FilterExec, LogicalRDD, QueryExecution, WholeStageCodegenExec} | ||
| import org.apache.spark.sql.execution.{FilterExec, LogicalRDD, QueryExecution, SortExec, WholeStageCodegenExec} | ||
| import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper | ||
| import org.apache.spark.sql.execution.aggregate.HashAggregateExec | ||
| import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, ReusedExchangeExec, ShuffleExchangeExec} | ||
| import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, ReusedExchangeExec, ShuffleExchangeExec, ShuffleExchangeLike} | ||
| import org.apache.spark.sql.expressions.{Aggregator, Window} | ||
| import org.apache.spark.sql.functions._ | ||
| import org.apache.spark.sql.internal.SQLConf | ||
|
|
@@ -3513,6 +3513,25 @@ class DataFrameSuite extends QueryTest | |
| assert(df.queryExecution.executedPlan.execute().getNumPartitions == 2) | ||
| } | ||
| } | ||
|
|
||
| test("SPARK-41048: Improve output partitioning and ordering with AQE cache") { | ||
| withSQLConf( | ||
| SQLConf.CAN_CHANGE_CACHED_PLAN_OUTPUT_PARTITIONING.key -> "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. after this PR, we can probably turn this on by default, to improve AQE coverage.
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. agree, We can also remvoe the |
||
| SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { | ||
| val df1 = spark.range(10).selectExpr("cast(id as string) c1") | ||
| val df2 = spark.range(10).selectExpr("cast(id as string) c2") | ||
| val cached = df1.join(df2, $"c1" === $"c2").cache() | ||
| cached.count() | ||
| val executedPlan = cached.groupBy("c1").agg(max($"c2")).queryExecution.executedPlan | ||
| // before is 2 sort and 1 shuffle | ||
| assert(collect(executedPlan) { | ||
| case s: ShuffleExchangeLike => s | ||
| }.isEmpty) | ||
| assert(collect(executedPlan) { | ||
| case s: SortExec => s | ||
| }.isEmpty) | ||
| } | ||
| } | ||
| } | ||
|
|
||
| case class GroupByKey(a: Int, b: Int) | ||
|
|
||
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
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 trying to understand this "best effort". AFAIK, table cache is lazy. For a query that accesses a cached query the first time, the cached query is not executed yet so we don't know the output partitioning/ordering and can't optimize out shuffles. But when the cached query is accessed the next time, it's already executed and we know the output partitioning/ordering.
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.
yes, in general the first action for a cached plan is
count, e.g.CacheTableAsSelectExec, so I think it is a not big issue that we can not optimize the shuffle/sort for the first action.The usage of the cache is: user wants to reference it multi-times, then this optimization will help a lot.
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.
This would be super limited use... and cause inconsistency.
I'd only return output partitioning if there is a user repartition op in the end. In other words, only if AQE plan is required to preserve user specified partitioning.
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.
Unfortunately we hint this.. per my experience, user always caches an arbitrary df and use the cached df to build an another arbitrary df. So why can't we preserve the partitioning/ordering of the cached plan ? If you really feel inconsistency in AdaptiveSparkPlanExec, we can probably move to InMemoryRelationExec.
My original idea is to do the both but feel a little overkill (requiredOrdering should be inferred separately like #35924)
requiredDistribution.map(_.createPartitioning(conf.shufflePartitions)).getOrElse { if (isFinalPlan) { executedPlan.outputPartitioning } else { super.outputPartitioning } }A useful distribution before caching is few in production since
repartition(col)will introduce skew