Skip to content

Commit 3c40be2

Browse files
committed
[SPARK-41405][SQL] Centralize the column resolution logic
### What changes were proposed in this pull request? This PR is a major refactor of how Spark resolves columns. Today, the column resolution logic is placed in several rules, which makes it hard to understand. It's also very fragile to maintain the resolution precedence, as you have to carefully deal with the interactions between these rules. This PR centralizes the column resolution logic into a single rule: the existing `ResolveReferences` rule, so that we no longer need to worry about the interactions between multiple rules. The detailed resolution precedence is also documented. ### Why are the changes needed? code cleanup ### Does this PR introduce _any_ user-facing change? no ### How was this patch tested? existing tests Closes #38888 from cloud-fan/col. Authored-by: Wenchen Fan <[email protected]> Signed-off-by: Wenchen Fan <[email protected]>
1 parent f0d9692 commit 3c40be2

File tree

8 files changed

+424
-453
lines changed

8 files changed

+424
-453
lines changed

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala

Lines changed: 386 additions & 411 deletions
Large diffs are not rendered by default.

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveLateralColumnAliasReference.scala

Lines changed: 6 additions & 18 deletions
Original file line numberDiff line numberDiff line change
@@ -21,8 +21,7 @@ import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeMap
2121
import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
2222
import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, LogicalPlan, Project}
2323
import org.apache.spark.sql.catalyst.rules.Rule
24-
import org.apache.spark.sql.catalyst.trees.TreeNodeTag
25-
import org.apache.spark.sql.catalyst.trees.TreePattern.LATERAL_COLUMN_ALIAS_REFERENCE
24+
import org.apache.spark.sql.catalyst.trees.TreePattern.{LATERAL_COLUMN_ALIAS_REFERENCE, TEMP_RESOLVED_COLUMN}
2625
import org.apache.spark.sql.catalyst.util.toPrettySQL
2726
import org.apache.spark.sql.errors.QueryCompilationErrors
2827
import org.apache.spark.sql.internal.SQLConf
@@ -82,26 +81,10 @@ import org.apache.spark.sql.internal.SQLConf
8281
* +- Aggregate [dept#14] [avg(salary#16) AS avg(salary)#26, avg(bonus#17) AS avg(bonus)#27,
8382
* dept#14]
8483
* +- Child [dept#14,name#15,salary#16,bonus#17]
85-
*
86-
*
87-
* The name resolution priority:
88-
* local table column > local lateral column alias > outer reference
89-
*
90-
* Because lateral column alias has higher resolution priority than outer reference, it will try
91-
* to resolve an [[OuterReference]] using lateral column alias in phase 1, similar as an
92-
* [[UnresolvedAttribute]]. If success, it strips [[OuterReference]] and also wraps it with
93-
* [[LateralColumnAliasReference]].
9484
*/
9585
object ResolveLateralColumnAliasReference extends Rule[LogicalPlan] {
9686
case class AliasEntry(alias: Alias, index: Int)
9787

98-
/**
99-
* A tag to store the nameParts from the original unresolved attribute.
100-
* It is set for [[OuterReference]], used in the current rule to convert [[OuterReference]] back
101-
* to [[LateralColumnAliasReference]].
102-
*/
103-
val NAME_PARTS_FROM_UNRESOLVED_ATTR = TreeNodeTag[Seq[String]]("name_parts_from_unresolved_attr")
104-
10588
private def assignAlias(expr: Expression): NamedExpression = {
10689
expr match {
10790
case ne: NamedExpression => ne
@@ -112,6 +95,11 @@ object ResolveLateralColumnAliasReference extends Rule[LogicalPlan] {
11295
override def apply(plan: LogicalPlan): LogicalPlan = {
11396
if (!conf.getConf(SQLConf.LATERAL_COLUMN_ALIAS_IMPLICIT_ENABLED)) {
11497
plan
98+
} else if (plan.containsPattern(TEMP_RESOLVED_COLUMN)) {
99+
// We should not change the plan if `TempResolvedColumn` is present in the query plan. It
100+
// needs certain plan shape to get resolved, such as Filter/Sort + Aggregate. LCA resolution
101+
// may break the plan shape, like adding Project above Aggregate.
102+
plan
115103
} else {
116104
// phase 2: unwrap
117105
plan.resolveOperatorsUpWithPruning(

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala

Lines changed: 16 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -661,13 +661,26 @@ case object UnresolvedSeed extends LeafExpression with Unevaluable {
661661

662662
/**
663663
* An intermediate expression to hold a resolved (nested) column. Some rules may need to undo the
664-
* column resolution and use this expression to keep the original column name.
664+
* column resolution and use this expression to keep the original column name, or redo the column
665+
* resolution with a different priority if the analyzer has tried to resolve it with the default
666+
* priority before but failed (i.e. `hasTried` is true).
665667
*/
666-
case class TempResolvedColumn(child: Expression, nameParts: Seq[String]) extends UnaryExpression
668+
case class TempResolvedColumn(
669+
child: Expression,
670+
nameParts: Seq[String],
671+
hasTried: Boolean = false) extends UnaryExpression
667672
with Unevaluable {
673+
// If it has been tried to be resolved but failed, mark it as unresolved so that other rules can
674+
// try to resolve it again.
675+
override lazy val resolved = child.resolved && !hasTried
668676
override lazy val canonicalized = child.canonicalized
669677
override def dataType: DataType = child.dataType
678+
override def nullable: Boolean = child.nullable
679+
// `TempResolvedColumn` is logically a leaf node. We should not count it as a missing reference
680+
// when resolving Filter/Sort/RepartitionByExpression. However, we should not make it a real
681+
// leaf node, as rules that update expr IDs should update `TempResolvedColumn.child` as well.
682+
override def references: AttributeSet = AttributeSet.empty
670683
override protected def withNewChildInternal(newChild: Expression): Expression =
671684
copy(child = newChild)
672-
override def sql: String = child.sql
685+
final override val nodePatterns: Seq[TreePattern] = Seq(TEMP_RESOLVED_COLUMN)
673686
}

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala

Lines changed: 12 additions & 11 deletions
Original file line numberDiff line numberDiff line change
@@ -433,26 +433,27 @@ case class OuterReference(e: NamedExpression)
433433

434434
/**
435435
* A placeholder used to hold a [[NamedExpression]] that has been temporarily resolved as the
436-
* reference to a lateral column alias.
436+
* reference to a lateral column alias. It will be restored back to [[UnresolvedAttribute]] if
437+
* the lateral column alias can't be resolved, or become a normal resolved column in the rewritten
438+
* plan after lateral column resolution. There should be no [[LateralColumnAliasReference]] beyond
439+
* analyzer: if the plan passes all analysis check, then all [[LateralColumnAliasReference]] should
440+
* already be removed.
437441
*
438-
* This is created and removed by Analyzer rule [[ResolveLateralColumnAlias]].
439-
* There should be no [[LateralColumnAliasReference]] beyond analyzer: if the plan passes all
440-
* analysis check, then all [[LateralColumnAliasReference]] should already be removed.
441-
*
442-
* @param ne the resolved [[NamedExpression]] by lateral column alias
443-
* @param nameParts the named parts of the original [[UnresolvedAttribute]]. Used to restore back
442+
* @param ne the [[NamedExpression]] produced by column resolution. Can be [[UnresolvedAttribute]]
443+
* if the referenced lateral column alias is not resolved yet.
444+
* @param nameParts the name parts of the original [[UnresolvedAttribute]]. Used to restore back
444445
* to [[UnresolvedAttribute]] when needed
445446
* @param a the attribute of referenced lateral column alias. Used to match alias when unwrapping
446-
* and resolving LateralColumnAliasReference
447+
* and resolving lateral column aliases and rewriting the query plan.
447448
*/
448449
case class LateralColumnAliasReference(ne: NamedExpression, nameParts: Seq[String], a: Attribute)
449450
extends LeafExpression with NamedExpression with Unevaluable {
450-
assert(ne.resolved)
451-
override def name: String =
452-
nameParts.map(n => if (n.contains(".")) s"`$n`" else n).mkString(".")
451+
assert(ne.resolved || ne.isInstanceOf[UnresolvedAttribute])
452+
override def name: String = ne.name
453453
override def exprId: ExprId = ne.exprId
454454
override def qualifier: Seq[String] = ne.qualifier
455455
override def toAttribute: Attribute = ne.toAttribute
456+
override lazy val resolved = ne.resolved
456457
override def newInstance(): NamedExpression =
457458
LateralColumnAliasReference(ne.newInstance(), nameParts, a)
458459

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala

Lines changed: 2 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -19,7 +19,6 @@ package org.apache.spark.sql.catalyst.expressions
1919

2020
import scala.collection.mutable.ArrayBuffer
2121

22-
import org.apache.spark.sql.catalyst.analysis.ResolveLateralColumnAliasReference.NAME_PARTS_FROM_UNRESOLVED_ATTR
2322
import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
2423
import org.apache.spark.sql.catalyst.plans.QueryPlan
2524
import org.apache.spark.sql.catalyst.plans.logical.{Filter, HintInfo, LogicalPlan}
@@ -159,12 +158,8 @@ object SubExprUtils extends PredicateHelper {
159158
/**
160159
* Wrap attributes in the expression with [[OuterReference]]s.
161160
*/
162-
def wrapOuterReference[E <: Expression](e: E, nameParts: Option[Seq[String]] = None): E = {
163-
e.transform { case a: Attribute =>
164-
val o = OuterReference(a)
165-
nameParts.map(o.setTagValue(NAME_PARTS_FROM_UNRESOLVED_ATTR, _))
166-
o
167-
}.asInstanceOf[E]
161+
def wrapOuterReference[E <: Expression](e: E): E = {
162+
e.transform { case a: Attribute => OuterReference(a) }.asInstanceOf[E]
168163
}
169164

170165
/**

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleIdCollection.scala

Lines changed: 0 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -58,7 +58,6 @@ object RuleIdCollection {
5858
"org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveGroupingAnalytics" ::
5959
"org.apache.spark.sql.catalyst.analysis.ResolveHigherOrderFunctions" ::
6060
"org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveInsertInto" ::
61-
"org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveMissingReferences" ::
6261
"org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveNaturalAndUsingJoin" ::
6362
"org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveNewInstance" ::
6463
"org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveOrdinalInOrderByAndGroupBy" ::

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreePatterns.scala

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -122,6 +122,7 @@ object TreePattern extends Enumeration {
122122
val UNION: Value = Value
123123
val UNRESOLVED_RELATION: Value = Value
124124
val UNRESOLVED_WITH: Value = Value
125+
val TEMP_RESOLVED_COLUMN: Value = Value
125126
val TYPED_FILTER: Value = Value
126127
val WINDOW: Value = Value
127128
val WITH_WINDOW_DEFINITION: Value = Value

sql/core/src/test/scala/org/apache/spark/sql/LateralColumnAliasSuite.scala

Lines changed: 1 addition & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -547,8 +547,7 @@ class LateralColumnAliasSuite extends LateralColumnAliasSuiteBase {
547547

548548
test("Lateral alias of a complex type") {
549549
// test both Project and Aggregate
550-
// TODO(anchovyu): re-enable aggregate tests when fixed the having issue
551-
val querySuffixes = Seq(""/* , s"FROM $testTable GROUP BY dept HAVING dept = 6" */)
550+
val querySuffixes = Seq("", s"FROM $testTable GROUP BY dept HAVING dept = 6")
552551
querySuffixes.foreach { querySuffix =>
553552
checkAnswer(
554553
sql(s"SELECT named_struct('a', 1) AS foo, foo.a + 1 AS bar, bar + 1 $querySuffix"),

0 commit comments

Comments
 (0)