-
Notifications
You must be signed in to change notification settings - Fork 28.9k
[SPARK-38578][SQL] AdaptiveSparkPlanExec should ensure user-specified ordering #35924
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 all commits
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 |
|---|---|---|
|
|
@@ -30,7 +30,7 @@ import org.apache.spark.broadcast | |
| import org.apache.spark.rdd.RDD | ||
| import org.apache.spark.sql.SparkSession | ||
| import org.apache.spark.sql.catalyst.InternalRow | ||
| import org.apache.spark.sql.catalyst.expressions.Attribute | ||
| import org.apache.spark.sql.catalyst.expressions.{Attribute, SortOrder} | ||
| import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, ReturnAnswer} | ||
| import org.apache.spark.sql.catalyst.plans.physical.{Distribution, UnspecifiedDistribution} | ||
| import org.apache.spark.sql.catalyst.rules.{PlanChangeLogger, Rule} | ||
|
|
@@ -97,6 +97,13 @@ case class AdaptiveSparkPlanExec( | |
| AQEUtils.getRequiredDistribution(inputPlan) | ||
| } | ||
|
|
||
| // Make sure AQE does not change the user-specified output ordering | ||
| @transient private val requiredOrdering: Seq[SortOrder] = if (isSubquery) { | ||
| Nil | ||
| } else { | ||
| AQEUtils.getRequiredOrdering(inputPlan) | ||
| } | ||
|
|
||
| @transient private val costEvaluator = | ||
| conf.getConf(SQLConf.ADAPTIVE_CUSTOM_COST_EVALUATOR_CLASS) match { | ||
| case Some(className) => CostEvaluator.instantiate(className, session.sparkContext.getConf) | ||
|
|
@@ -112,7 +119,7 @@ case class AdaptiveSparkPlanExec( | |
| // `EnsureRequirements` to not optimize out the user-specified repartition-by-col to work | ||
| // around this case. | ||
| val ensureRequirements = | ||
| EnsureRequirements(requiredDistribution.isDefined, requiredDistribution) | ||
| EnsureRequirements(requiredDistribution.isDefined, requiredDistribution, requiredOrdering) | ||
| Seq( | ||
| RemoveRedundantProjects, | ||
| ensureRequirements, | ||
|
|
@@ -163,7 +170,7 @@ case class AdaptiveSparkPlanExec( | |
| } else { | ||
| UnspecifiedDistribution | ||
| } | ||
| if (ValidateRequirements.validate(applied, distribution)) { | ||
| if (ValidateRequirements.validate(applied, distribution, requiredOrdering)) { | ||
| applied | ||
| } else { | ||
| logDebug(s"Rule ${rule.ruleName} is not applied as it breaks the " + | ||
|
|
@@ -207,6 +214,8 @@ case class AdaptiveSparkPlanExec( | |
|
|
||
| override def output: Seq[Attribute] = inputPlan.output | ||
|
|
||
| override def outputOrdering: Seq[SortOrder] = requiredOrdering | ||
|
||
|
|
||
| override def doCanonicalize(): SparkPlan = inputPlan.canonicalized | ||
|
|
||
| override def resetMetrics(): Unit = { | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -30,6 +30,14 @@ import org.apache.spark.sql.execution._ | |
| */ | ||
| object ValidateRequirements extends Logging { | ||
|
|
||
| def validate( | ||
| plan: SparkPlan, | ||
| requiredDistribution: Distribution, | ||
| requiredOrdering: Seq[SortOrder]): Boolean = { | ||
| validate(plan, requiredDistribution) && | ||
| SortOrder.orderingSatisfies(plan.outputOrdering, requiredOrdering) | ||
| } | ||
|
|
||
| def validate(plan: SparkPlan, requiredDistribution: Distribution): Boolean = { | ||
|
||
| validate(plan) && plan.outputPartitioning.satisfies(requiredDistribution) | ||
| } | ||
|
|
||
Uh oh!
There was an error while loading. Please reload this page.