Skip to content

Commit

Permalink
[SPARK-49556][SQL] Add SQL pipe syntax for the SELECT operator
Browse files Browse the repository at this point in the history
### What changes were proposed in this pull request?

This PR adds SQL pipe syntax support for the SELECT operator.

For example:

```
CREATE TABLE t(x INT, y STRING) USING CSV;
INSERT INTO t VALUES (0, 'abc'), (1, 'def');

TABLE t
|> SELECT x, y

0	abc
1	def

TABLE t
|> SELECT x, y
|> SELECT x + LENGTH(y) AS z

3
4

(SELECT * FROM t UNION ALL SELECT * FROM t)
|> SELECT x + LENGTH(y) AS result

3
3
4
4

TABLE t
|> SELECT sum(x) AS result

Error: aggregate functions are not allowed in the pipe operator |> SELECT clause; please use the |> AGGREGATE clause instead
```

### Why are the changes needed?

The SQL pipe operator syntax will let users compose queries in a more flexible fashion.

### Does this PR introduce _any_ user-facing change?

Yes, see above.

### How was this patch tested?

This PR adds a few unit test cases, but mostly relies on golden file test coverage. I did this to make sure the answers are correct as this feature is implemented and also so we can look at the analyzer output plans to ensure they look right as well.

### Was this patch authored or co-authored using generative AI tooling?

No

Closes #48047 from dtenedor/pipe-select.

Authored-by: Daniel Tenedorio <daniel.tenedorio@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
  • Loading branch information
dtenedor authored and cloud-fan committed Sep 14, 2024
1 parent 2250b35 commit 017b0ea
Show file tree
Hide file tree
Showing 13 changed files with 876 additions and 9 deletions.
6 changes: 6 additions & 0 deletions common/utils/src/main/resources/error/error-conditions.json
Original file line number Diff line number Diff line change
Expand Up @@ -3754,6 +3754,12 @@
],
"sqlState" : "42K03"
},
"PIPE_OPERATOR_SELECT_CONTAINS_AGGREGATE_FUNCTION" : {
"message" : [
"Aggregate function <expr> is not allowed when using the pipe operator |> SELECT clause; please use the pipe operator |> AGGREGATE clause instead"
],
"sqlState" : "0A000"
},
"PIVOT_VALUE_DATA_TYPE_MISMATCH" : {
"message" : [
"Invalid pivot value '<value>': value data type <valueType> does not match pivot column data type <pivotType>."
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -506,6 +506,7 @@ TILDE: '~';
AMPERSAND: '&';
PIPE: '|';
CONCAT_PIPE: '||';
OPERATOR_PIPE: '|>';
HAT: '^';
COLON: ':';
DOUBLE_COLON: '::';
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -613,6 +613,7 @@ queryTerm
operator=INTERSECT setQuantifier? right=queryTerm #setOperation
| left=queryTerm {!legacy_setops_precedence_enabled}?
operator=(UNION | EXCEPT | SETMINUS) setQuantifier? right=queryTerm #setOperation
| left=queryTerm OPERATOR_PIPE operatorPipeRightSide #operatorPipeStatement
;

queryPrimary
Expand Down Expand Up @@ -1471,6 +1472,10 @@ version
| stringLit
;

operatorPipeRightSide
: selectClause
;

// When `SQL_standard_keyword_behavior=true`, there are 2 kinds of keywords in Spark SQL.
// - Reserved keywords:
// Keywords that are reserved and can't be used as identifiers for table, view, column,
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,47 @@
/*
* 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.expressions

import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateFunction
import org.apache.spark.sql.catalyst.trees.TreePattern.{PIPE_OPERATOR_SELECT, RUNTIME_REPLACEABLE, TreePattern}
import org.apache.spark.sql.errors.QueryCompilationErrors

/**
* Represents a SELECT clause when used with the |> SQL pipe operator.
* We use this to make sure that no aggregate functions exist in the SELECT expressions.
*/
case class PipeSelect(child: Expression)
extends UnaryExpression with RuntimeReplaceable {
final override val nodePatterns: Seq[TreePattern] = Seq(PIPE_OPERATOR_SELECT, RUNTIME_REPLACEABLE)
override def withNewChildInternal(newChild: Expression): Expression = PipeSelect(newChild)
override lazy val replacement: Expression = {
def visit(e: Expression): Unit = e match {
case a: AggregateFunction =>
// If we used the pipe operator |> SELECT clause to specify an aggregate function, this is
// invalid; return an error message instructing the user to use the pipe operator
// |> AGGREGATE clause for this purpose instead.
throw QueryCompilationErrors.pipeOperatorSelectContainsAggregateFunction(a)
case _: WindowExpression =>
// Window functions are allowed in pipe SELECT operators, so do not traverse into children.
case _ =>
e.children.foreach(visit)
}
visit(child)
child
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -469,7 +469,8 @@ class AstBuilder extends DataTypeAstBuilder
ctx.aggregationClause,
ctx.havingClause,
ctx.windowClause,
plan
plan,
isPipeOperatorSelect = false
)
}
}
Expand Down Expand Up @@ -1057,7 +1058,8 @@ class AstBuilder extends DataTypeAstBuilder
ctx.aggregationClause,
ctx.havingClause,
ctx.windowClause,
from
from,
isPipeOperatorSelect = false
)
}

Expand Down Expand Up @@ -1144,7 +1146,8 @@ class AstBuilder extends DataTypeAstBuilder
aggregationClause,
havingClause,
windowClause,
isDistinct = false)
isDistinct = false,
isPipeOperatorSelect = false)

ScriptTransformation(
string(visitStringLit(transformClause.script)),
Expand All @@ -1165,6 +1168,8 @@ class AstBuilder extends DataTypeAstBuilder
* Add a regular (SELECT) query specification to a logical plan. The query specification
* is the core of the logical plan, this is where sourcing (FROM clause), projection (SELECT),
* aggregation (GROUP BY ... HAVING ...) and filtering (WHERE) takes place.
* If 'isPipeOperatorSelect' is true, wraps each projected expression with a [[PipeSelect]]
* expression for future validation of the expressions during analysis.
*
* Note that query hints are ignored (both by the parser and the builder).
*/
Expand All @@ -1176,7 +1181,8 @@ class AstBuilder extends DataTypeAstBuilder
aggregationClause: AggregationClauseContext,
havingClause: HavingClauseContext,
windowClause: WindowClauseContext,
relation: LogicalPlan): LogicalPlan = withOrigin(ctx) {
relation: LogicalPlan,
isPipeOperatorSelect: Boolean): LogicalPlan = withOrigin(ctx) {
val isDistinct = selectClause.setQuantifier() != null &&
selectClause.setQuantifier().DISTINCT() != null

Expand All @@ -1188,7 +1194,8 @@ class AstBuilder extends DataTypeAstBuilder
aggregationClause,
havingClause,
windowClause,
isDistinct)
isDistinct,
isPipeOperatorSelect)

// Hint
selectClause.hints.asScala.foldRight(plan)(withHints)
Expand All @@ -1202,7 +1209,8 @@ class AstBuilder extends DataTypeAstBuilder
aggregationClause: AggregationClauseContext,
havingClause: HavingClauseContext,
windowClause: WindowClauseContext,
isDistinct: Boolean): LogicalPlan = {
isDistinct: Boolean,
isPipeOperatorSelect: Boolean): LogicalPlan = {
// Add lateral views.
val withLateralView = lateralView.asScala.foldLeft(relation)(withGenerate)

Expand All @@ -1216,7 +1224,20 @@ class AstBuilder extends DataTypeAstBuilder
}

def createProject() = if (namedExpressions.nonEmpty) {
Project(namedExpressions, withFilter)
val newProjectList: Seq[NamedExpression] = if (isPipeOperatorSelect) {
// If this is a pipe operator |> SELECT clause, add a [[PipeSelect]] expression wrapping
// each alias in the project list, so the analyzer can check invariants later.
namedExpressions.map {
case a: Alias =>
a.withNewChildren(Seq(PipeSelect(a.child)))
.asInstanceOf[NamedExpression]
case other =>
other
}
} else {
namedExpressions
}
Project(newProjectList, withFilter)
} else {
withFilter
}
Expand Down Expand Up @@ -5755,6 +5776,29 @@ class AstBuilder extends DataTypeAstBuilder
visitSetVariableImpl(ctx.query(), ctx.multipartIdentifierList(), ctx.assignmentList())
}

override def visitOperatorPipeStatement(ctx: OperatorPipeStatementContext): LogicalPlan = {
visitOperatorPipeRightSide(ctx.operatorPipeRightSide(), plan(ctx.left))
}

private def visitOperatorPipeRightSide(
ctx: OperatorPipeRightSideContext, left: LogicalPlan): LogicalPlan = {
if (!SQLConf.get.getConf(SQLConf.OPERATOR_PIPE_SYNTAX_ENABLED)) {
operationNotAllowed("Operator pipe SQL syntax using |>", ctx)
}
Option(ctx.selectClause).map { c =>
withSelectQuerySpecification(
ctx = ctx,
selectClause = c,
lateralView = new java.util.ArrayList[LateralViewContext](),
whereClause = null,
aggregationClause = null,
havingClause = null,
windowClause = null,
relation = left,
isPipeOperatorSelect = true)
}.get
}

/**
* Check plan for any parameters.
* If it finds any throws UNSUPPORTED_FEATURE.PARAMETER_MARKER_IN_UNEXPECTED_STATEMENT.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -72,6 +72,7 @@ object TreePattern extends Enumeration {
val NOT: Value = Value
val NULL_CHECK: Value = Value
val NULL_LITERAL: Value = Value
val PIPE_OPERATOR_SELECT: Value = Value
val SERIALIZE_FROM_OBJECT: Value = Value
val OR: Value = Value
val OUTER_REFERENCE: Value = Value
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -4104,4 +4104,12 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase with Compilat
messageParameters = Map("functionName" -> functionName)
)
}

def pipeOperatorSelectContainsAggregateFunction(expr: Expression): Throwable = {
new AnalysisException(
errorClass = "PIPE_OPERATOR_SELECT_CONTAINS_AGGREGATE_FUNCTION",
messageParameters = Map(
"expr" -> expr.toString),
origin = expr.origin)
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -5012,6 +5012,15 @@ object SQLConf {
.stringConf
.createWithDefault("versionAsOf")

val OPERATOR_PIPE_SYNTAX_ENABLED =
buildConf("spark.sql.operatorPipeSyntaxEnabled")
.doc("If true, enable operator pipe syntax for Apache Spark SQL. This uses the operator " +
"pipe marker |> to indicate separation between clauses of SQL in a manner that describes " +
"the sequence of steps that the query performs in a composable fashion.")
.version("4.0.0")
.booleanConf
.createWithDefault(Utils.isTesting)

val LEGACY_PERCENTILE_DISC_CALCULATION = buildConf("spark.sql.legacy.percentileDiscCalculation")
.internal()
.doc("If true, the old bogus percentile_disc calculation is used. The old calculation " +
Expand Down
Loading

0 comments on commit 017b0ea

Please sign in to comment.