-
Notifications
You must be signed in to change notification settings - Fork 28.5k
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[SPARK-27561][SQL] Support implicit lateral column alias resolution o…
…n Project ### What changes were proposed in this pull request? This PR implements a new feature: Implicit lateral column alias on `Project` case, controlled by `spark.sql.lateralColumnAlias.enableImplicitResolution` temporarily (default false now, but will turn on this conf once the feature is completely merged). #### Lateral column alias View https://issues.apache.org/jira/browse/SPARK-27561 for more details on lateral column alias. There are two main cases to support: LCA in Project, and LCA in Aggregate. ```sql -- LCA in Project. The base_salary references an attribute defined by a previous alias SELECT salary AS base_salary, base_salary + bonus AS total_salary FROM employee -- LCA in Aggregate. The avg_salary references an attribute defined by a previous alias SELECT dept, average(salary) AS avg_salary, avg_salary + average(bonus) FROM employee GROUP BY dept ``` This **implicit** lateral column alias (no explicit keyword, e.g. `lateral.base_salary`) should be supported. #### High level design This PR defines a new Resolution rule, `ResolveLateralColumnAlias` to resolve the implicit lateral column alias, covering the `Project` case. It introduces a new leaf node NamedExpression, `LateralColumnAliasReference`, as a placeholder used to hold a referenced that has been temporarily resolved as the reference to a lateral column alias. The whole process is generally divided into two phases: 1) recognize **resolved** lateral alias, wrap the attributes referencing them with `LateralColumnAliasReference`. 2) when the whole operator is resolved, unwrap `LateralColumnAliasReference`. For Project, it further resolves the attributes and push down the referenced lateral aliases to the new Project. For example: ``` // Before Project [age AS a, 'a + 1] +- Child // After phase 1 Project [age AS a, lateralalias(a) + 1] +- Child // After phase 2 Project [a, a + 1] +- Project [child output, age AS a] +- Child ``` #### Resolution order Given this new rule, the name resolution order will be (higher -> lower): ``` local table column > local metadata attribute > local lateral column alias > all others (outer reference of subquery, parameters of SQL UDF, ..) ``` There is a recent refactor that moves the creation of `OuterReference` in the Resolution batch: #38851. Because lateral column alias has higher resolution priority than outer reference, it will try to resolve an `OuterReference` using lateral column alias, similar as an `UnresolvedAttribute`. If success, it strips `OuterReference` and also wraps it with `LateralColumnAliasReference`. ### Why are the changes needed? The lateral column alias is a popular feature wanted for a long time. It is supported by lots of other database vendors (Redshift, snowflake, etc) and provides a better user experience. ### Does this PR introduce _any_ user-facing change? Yes, as shown in the above example, it will be able to resolve lateral column alias. I will write the migration guide or release note when most PRs of this feature are merged. ### How was this patch tested? Existing tests and newly added tests. Closes #38776 from anchovYu/SPARK-27561-refactor. Authored-by: Xinyi Yu <xinyi.yu@databricks.com> Signed-off-by: Wenchen Fan <wenchen@databricks.com>
- Loading branch information
Showing
13 changed files
with
686 additions
and
7 deletions.
There are no files selected for viewing
This file contains 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 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 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 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 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
135 changes: 135 additions & 0 deletions
135
...ain/scala/org/apache/spark/sql/catalyst/analysis/ResolveLateralColumnAliasReference.scala
This file contains 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 |
---|---|---|
@@ -0,0 +1,135 @@ | ||
/* | ||
* Licensed to the Apache Software Foundation (ASF) under one or more | ||
* contributor license agreements. See the NOTICE file distributed with | ||
* this work for additional information regarding copyright ownership. | ||
* The ASF licenses this file to You under the Apache License, Version 2.0 | ||
* (the "License"); you may not use this file except in compliance with | ||
* the License. You may obtain a copy of the License at | ||
* | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, | ||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
* See the License for the specific language governing permissions and | ||
* limitations under the License. | ||
*/ | ||
|
||
package org.apache.spark.sql.catalyst.analysis | ||
|
||
import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeMap, LateralColumnAliasReference, NamedExpression} | ||
import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project} | ||
import org.apache.spark.sql.catalyst.rules.Rule | ||
import org.apache.spark.sql.catalyst.trees.TreeNodeTag | ||
import org.apache.spark.sql.catalyst.trees.TreePattern.LATERAL_COLUMN_ALIAS_REFERENCE | ||
import org.apache.spark.sql.internal.SQLConf | ||
|
||
/** | ||
* This rule is the second phase to resolve lateral column alias. | ||
* | ||
* Resolve lateral column alias, which references the alias defined previously in the SELECT list. | ||
* Plan-wise, it handles two types of operators: Project and Aggregate. | ||
* - in Project, pushing down the referenced lateral alias into a newly created Project, resolve | ||
* the attributes referencing these aliases | ||
* - in Aggregate TODO. | ||
* | ||
* The whole process is generally divided into two phases: | ||
* 1) recognize resolved lateral alias, wrap the attributes referencing them with | ||
* [[LateralColumnAliasReference]] | ||
* 2) when the whole operator is resolved, unwrap [[LateralColumnAliasReference]]. | ||
* For Project, it further resolves the attributes and push down the referenced lateral aliases. | ||
* For Aggregate, TODO | ||
* | ||
* Example for Project: | ||
* Before rewrite: | ||
* Project [age AS a, 'a + 1] | ||
* +- Child | ||
* | ||
* After phase 1: | ||
* Project [age AS a, lateralalias(a) + 1] | ||
* +- Child | ||
* | ||
* After phase 2: | ||
* Project [a, a + 1] | ||
* +- Project [child output, age AS a] | ||
* +- Child | ||
* | ||
* Example for Aggregate TODO | ||
* | ||
* | ||
* The name resolution priority: | ||
* local table column > local lateral column alias > outer reference | ||
* | ||
* Because lateral column alias has higher resolution priority than outer reference, it will try | ||
* to resolve an [[OuterReference]] using lateral column alias in phase 1, similar as an | ||
* [[UnresolvedAttribute]]. If success, it strips [[OuterReference]] and also wraps it with | ||
* [[LateralColumnAliasReference]]. | ||
*/ | ||
object ResolveLateralColumnAliasReference extends Rule[LogicalPlan] { | ||
case class AliasEntry(alias: Alias, index: Int) | ||
|
||
/** | ||
* A tag to store the nameParts from the original unresolved attribute. | ||
* It is set for [[OuterReference]], used in the current rule to convert [[OuterReference]] back | ||
* to [[LateralColumnAliasReference]]. | ||
*/ | ||
val NAME_PARTS_FROM_UNRESOLVED_ATTR = TreeNodeTag[Seq[String]]("name_parts_from_unresolved_attr") | ||
|
||
override def apply(plan: LogicalPlan): LogicalPlan = { | ||
if (!conf.getConf(SQLConf.LATERAL_COLUMN_ALIAS_IMPLICIT_ENABLED)) { | ||
plan | ||
} else { | ||
// phase 2: unwrap | ||
plan.resolveOperatorsUpWithPruning( | ||
_.containsPattern(LATERAL_COLUMN_ALIAS_REFERENCE), ruleId) { | ||
case p @ Project(projectList, child) if p.resolved | ||
&& projectList.exists(_.containsPattern(LATERAL_COLUMN_ALIAS_REFERENCE)) => | ||
var aliasMap = AttributeMap.empty[AliasEntry] | ||
val referencedAliases = collection.mutable.Set.empty[AliasEntry] | ||
def unwrapLCAReference(e: NamedExpression): NamedExpression = { | ||
e.transformWithPruning(_.containsPattern(LATERAL_COLUMN_ALIAS_REFERENCE)) { | ||
case lcaRef: LateralColumnAliasReference if aliasMap.contains(lcaRef.a) => | ||
val aliasEntry = aliasMap.get(lcaRef.a).get | ||
// If there is no chaining of lateral column alias reference, push down the alias | ||
// and unwrap the LateralColumnAliasReference to the NamedExpression inside | ||
// If there is chaining, don't resolve and save to future rounds | ||
if (!aliasEntry.alias.containsPattern(LATERAL_COLUMN_ALIAS_REFERENCE)) { | ||
referencedAliases += aliasEntry | ||
lcaRef.ne | ||
} else { | ||
lcaRef | ||
} | ||
case lcaRef: LateralColumnAliasReference if !aliasMap.contains(lcaRef.a) => | ||
// It shouldn't happen, but restore to unresolved attribute to be safe. | ||
UnresolvedAttribute(lcaRef.nameParts) | ||
}.asInstanceOf[NamedExpression] | ||
} | ||
val newProjectList = projectList.zipWithIndex.map { | ||
case (a: Alias, idx) => | ||
val lcaResolved = unwrapLCAReference(a) | ||
// Insert the original alias instead of rewritten one to detect chained LCA | ||
aliasMap += (a.toAttribute -> AliasEntry(a, idx)) | ||
lcaResolved | ||
case (e, _) => | ||
unwrapLCAReference(e) | ||
} | ||
|
||
if (referencedAliases.isEmpty) { | ||
p | ||
} else { | ||
val outerProjectList = collection.mutable.Seq(newProjectList: _*) | ||
val innerProjectList = | ||
collection.mutable.ArrayBuffer(child.output.map(_.asInstanceOf[NamedExpression]): _*) | ||
referencedAliases.foreach { case AliasEntry(alias: Alias, idx) => | ||
outerProjectList.update(idx, alias.toAttribute) | ||
innerProjectList += alias | ||
} | ||
p.copy( | ||
projectList = outerProjectList.toSeq, | ||
child = Project(innerProjectList.toSeq, child) | ||
) | ||
} | ||
} | ||
} | ||
} | ||
} |
Oops, something went wrong.