dtenedor commented on code in PR #48649:
URL: https://github.com/apache/spark/pull/48649#discussion_r1824806683
##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala:
##########
@@ -5874,53 +5886,72 @@ class AstBuilder extends DataTypeAstBuilder
if (!SQLConf.get.getConf(SQLConf.OPERATOR_PIPE_SYNTAX_ENABLED)) {
operationNotAllowed("Operator pipe SQL syntax using |>", ctx)
}
- // This helper function adds a table subquery boundary between the new
operator to be added
- // (such as a filter or sort) and the input plan if one does not already
exist. This helps the
- // analyzer behave as if we had added the corresponding SQL clause after a
table subquery
- // containing the input plan.
- def withSubqueryAlias(): LogicalPlan = left match {
- case s: SubqueryAlias =>
- s
- case u: UnresolvedRelation =>
- u
- case _ =>
- SubqueryAlias(SubqueryAlias.generateSubqueryName(), left)
- }
- 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)
- }.getOrElse(Option(ctx.whereClause).map { c =>
- withWhereClause(c, withSubqueryAlias())
- }.getOrElse(Option(ctx.pivotClause()).map { c =>
- if (ctx.unpivotClause() != null) {
- throw
QueryParsingErrors.unpivotWithPivotInFromClauseNotAllowedError(ctx)
- }
- withPivot(c, left)
- }.getOrElse(Option(ctx.unpivotClause()).map { c =>
- if (ctx.pivotClause() != null) {
- throw
QueryParsingErrors.unpivotWithPivotInFromClauseNotAllowedError(ctx)
+
+ // Extract the base child and a list of WithWindowDefinition nodes from
the plan
+ var baseChild = left
+ var withWindowDefinitions = List.empty[WithWindowDefinition]
+ while (baseChild.isInstanceOf[WithWindowDefinition]) {
+ val wwd = baseChild.asInstanceOf[WithWindowDefinition]
+ withWindowDefinitions = withWindowDefinitions :+ wwd
Review Comment:
```suggestion
withWindowDefinitions += wwd
```
##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala:
##########
@@ -5874,53 +5886,72 @@ class AstBuilder extends DataTypeAstBuilder
if (!SQLConf.get.getConf(SQLConf.OPERATOR_PIPE_SYNTAX_ENABLED)) {
operationNotAllowed("Operator pipe SQL syntax using |>", ctx)
}
- // This helper function adds a table subquery boundary between the new
operator to be added
- // (such as a filter or sort) and the input plan if one does not already
exist. This helps the
- // analyzer behave as if we had added the corresponding SQL clause after a
table subquery
- // containing the input plan.
- def withSubqueryAlias(): LogicalPlan = left match {
- case s: SubqueryAlias =>
- s
- case u: UnresolvedRelation =>
- u
- case _ =>
- SubqueryAlias(SubqueryAlias.generateSubqueryName(), left)
- }
- 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)
- }.getOrElse(Option(ctx.whereClause).map { c =>
- withWhereClause(c, withSubqueryAlias())
- }.getOrElse(Option(ctx.pivotClause()).map { c =>
- if (ctx.unpivotClause() != null) {
- throw
QueryParsingErrors.unpivotWithPivotInFromClauseNotAllowedError(ctx)
- }
- withPivot(c, left)
- }.getOrElse(Option(ctx.unpivotClause()).map { c =>
- if (ctx.pivotClause() != null) {
- throw
QueryParsingErrors.unpivotWithPivotInFromClauseNotAllowedError(ctx)
+
+ // Extract the base child and a list of WithWindowDefinition nodes from
the plan
+ var baseChild = left
+ var withWindowDefinitions = List.empty[WithWindowDefinition]
+ while (baseChild.isInstanceOf[WithWindowDefinition]) {
+ val wwd = baseChild.asInstanceOf[WithWindowDefinition]
+ withWindowDefinitions = withWindowDefinitions :+ wwd
+ baseChild = wwd.child
+ }
+
+ // Process the base child
+ val newChild = {
+ // This helper function adds a table subquery boundary between the new
operator to be added
+ // (such as a filter or sort) and the input plan if one does not already
exist. This helps the
+ // analyzer behave as if we had added the corresponding SQL clause after
a table subquery
+ // containing the input plan.
+ def withSubqueryAlias(): LogicalPlan = baseChild match {
+ case s: SubqueryAlias =>
+ s
+ case u: UnresolvedRelation =>
+ u
+ case _ =>
+ SubqueryAlias(SubqueryAlias.generateSubqueryName(), baseChild)
}
- withUnpivot(c, left)
- }.getOrElse(Option(ctx.sample).map { c =>
- withSample(c, left)
- }.getOrElse(Option(ctx.joinRelation()).map { c =>
- withJoinRelation(c, left)
- }.getOrElse(Option(ctx.operator).map { c =>
- val all = Option(ctx.setQuantifier()).exists(_.ALL != null)
- visitSetOperationImpl(left, plan(ctx.right), all, c.getType)
- }.getOrElse(Option(ctx.queryOrganization).map { c =>
- withQueryResultClauses(c, withSubqueryAlias(), forPipeOperators = true)
- }.getOrElse(
- visitOperatorPipeAggregate(ctx, left)
- ))))))))
+
+ 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 = baseChild,
+ isPipeOperatorSelect = true)
+ }.getOrElse(Option(ctx.whereClause).map { c =>
+ withWhereClause(c, withSubqueryAlias())
+ }.getOrElse(Option(ctx.pivotClause()).map { c =>
+ if (ctx.unpivotClause() != null) {
+ throw
QueryParsingErrors.unpivotWithPivotInFromClauseNotAllowedError(ctx)
+ }
+ withPivot(c, baseChild)
+ }.getOrElse(Option(ctx.unpivotClause()).map { c =>
+ if (ctx.pivotClause() != null) {
+ throw
QueryParsingErrors.unpivotWithPivotInFromClauseNotAllowedError(ctx)
+ }
+ withUnpivot(c, baseChild)
+ }.getOrElse(Option(ctx.sample).map { c =>
+ withSample(c, baseChild)
+ }.getOrElse(Option(ctx.joinRelation()).map { c =>
+ withJoinRelation(c, baseChild)
+ }.getOrElse(Option(ctx.operator).map { c =>
+ val all = Option(ctx.setQuantifier()).exists(_.ALL != null)
+ visitSetOperationImpl(baseChild, plan(ctx.right), all, c.getType)
+ }.getOrElse(Option(ctx.queryOrganization).map { c =>
+ withQueryResultClauses(c, withSubqueryAlias(), forPipeOperators = true)
+ }.getOrElse(
+ visitOperatorPipeAggregate(ctx, left)
+ ))))))))
+ }
+
+ // Reconstruct the WithWindowDefinition nodes on top of the new child
Review Comment:
It looks like we don't allow multiple consecutive WINDOW clauses, as shown
by this new test in our PR [1]. Do we still need this logic of matching against
multiple chained `WithWindowDefinition` notes in the plan and then iterating
through that entire list at the end like this? Or would it be sufficient to
just check for a single `WithWindowDefinition` node instead?
Looking at this test failure, it seems it is actually complaining that the
window specification `w1` is no longer visible because it has been overwritten
by `w2`. It seems like we actually want to throw a more specific error at the
parse location of `window w2` in this case, reporting that multiple consecutive
WINDOW SQL pipe operators are not supported. (It makes sense to throw this
error because regular SQL syntax does not allow multiple consecutive WINDOW
clauses either.) Then we can simplify our logic here to only remember the
single `WithWindowDefinition` operator in the happy case.
[1]
```
-- !query
table windowTestData
|> window w1 as (partition by cate)
|> window w2 as (partition by val order by cate)
|> select sum(val) over w1, first_value(cate) over w2
-- !query schema
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
{
"errorClass" : "MISSING_WINDOW_SPECIFICATION",
"sqlState" : "42P20",
"messageParameters" : {
"docroot" : "https://spark.apache.org/docs/latest",
"windowName" : "w1"
}
}
```
##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala:
##########
@@ -5874,53 +5886,72 @@ class AstBuilder extends DataTypeAstBuilder
if (!SQLConf.get.getConf(SQLConf.OPERATOR_PIPE_SYNTAX_ENABLED)) {
operationNotAllowed("Operator pipe SQL syntax using |>", ctx)
}
- // This helper function adds a table subquery boundary between the new
operator to be added
- // (such as a filter or sort) and the input plan if one does not already
exist. This helps the
- // analyzer behave as if we had added the corresponding SQL clause after a
table subquery
- // containing the input plan.
- def withSubqueryAlias(): LogicalPlan = left match {
- case s: SubqueryAlias =>
- s
- case u: UnresolvedRelation =>
- u
- case _ =>
- SubqueryAlias(SubqueryAlias.generateSubqueryName(), left)
- }
- 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)
- }.getOrElse(Option(ctx.whereClause).map { c =>
- withWhereClause(c, withSubqueryAlias())
- }.getOrElse(Option(ctx.pivotClause()).map { c =>
- if (ctx.unpivotClause() != null) {
- throw
QueryParsingErrors.unpivotWithPivotInFromClauseNotAllowedError(ctx)
- }
- withPivot(c, left)
- }.getOrElse(Option(ctx.unpivotClause()).map { c =>
- if (ctx.pivotClause() != null) {
- throw
QueryParsingErrors.unpivotWithPivotInFromClauseNotAllowedError(ctx)
+
+ // Extract the base child and a list of WithWindowDefinition nodes from
the plan
+ var baseChild = left
+ var withWindowDefinitions = List.empty[WithWindowDefinition]
Review Comment:
```suggestion
val withWindowDefinitions = ArrayBuffer.empty[WithWindowDefinition]
```
This way the performance can be faster by not constructing a new list for
each append :)
##########
sql/core/src/test/resources/sql-tests/inputs/pipe-operators.sql:
##########
@@ -665,15 +665,119 @@ table t
table t
|> order by x sort by x;
--- The WINDOW clause is not supported yet.
+-- The WINDOW clause: positive tests.
+-------------------------------------
+-- WINDOW with one definition
table windowTestData
-|> window w as (partition by cte order by val)
+|> window w as (partition by cate order by val)
|> select cate, sum(val) over w;
+-- WINDOW with multiple definitions
+table windowTestData
+|> window w1 as (partition by cate), w2 as (partition by val order by cate)
+|> select sum(val) over w1, first_value(cate) over w2;
+
+-- WINDOW with RANGE BETWEEN as part of the window definition
+table windowTestData
+|> window w as (order by val_timestamp range between unbounded preceding and
current row)
+|> select avg(val) over w;
+
+-- WINDOW definition not being referred in the following SELECT clause
+table windowTestData
+|> window w as (partition by cate order by val)
+|> select cate;
+
+-- WINDOW definition not being referred in the following WHERE clause
+table windowTestData
+|> window w as (partition by cate order by val)
+|> where cate = "a";
+
+-- WINDOW definition not being referred in the following ORDER BY clause
+table windowTestData
+|> window w as (partition by cate order by val)
+|> order by cate;
+
+-- WINDOW referred in the following SELECT clause, with a WHERE after it
+table windowTestData
+|> window w as (partition by cate order by val)
+|> select cate, sum(val) over w
+|> where cate = "a";
+
+-- WINDOW referred in the following SELECT clause, with an ORDER BY after it
+table windowTestData
+|> window w as (partition by cate order by val)
+|> select cate, sum(val) over w as sum_val
+|> order by sum_val;
+
+-- WINDOW with table aliases to refer columns
+table windowTestData
+|> window w as (partition by windowTestData.cate order by windowTestData.val)
+|> select windowTestData.cate, sum(windowTestData.val) over w;
+
+-- WINDOW using struct fields.
+(select col from st)
+|> window w as (partition by col.i1 order by col.i2)
+|> select col.i1, sum(col.i2) over w;
+
+table st
+|> window w as (partition by st.col.i1 order by st.col.i2)
+|> select st.col.i1, sum(st.col.i2) over w;
+
+table st
+|> window w as (partition by spark_catalog.default.st.col.i1 order by
spark_catalog.default.st.col.i2)
+|> select spark_catalog.default.st.col.i1,
sum(spark_catalog.default.st.col.i2) over w;
+
+-- The WINDOW clause: debatable positive cases.
+------------------------------------------------
+-- WINDOW with nothing after it.
+table windowTestData
+|> window w as (partition by cate);
+
+table windowTestData
+|> window w as (partition by cate) order by val;
+
+-- WINDOW names can shadow other valid column names (align with the current
SQL syntax).
Review Comment:
If this is allowed with regular SQL syntax, it seems consistent to allow it
for pipe syntax too. We can move this to the end of the "positive tests"
section.
##########
sql/core/src/test/resources/sql-tests/inputs/pipe-operators.sql:
##########
@@ -665,15 +665,119 @@ table t
table t
|> order by x sort by x;
--- The WINDOW clause is not supported yet.
+-- The WINDOW clause: positive tests.
+-------------------------------------
+-- WINDOW with one definition
table windowTestData
-|> window w as (partition by cte order by val)
+|> window w as (partition by cate order by val)
|> select cate, sum(val) over w;
+-- WINDOW with multiple definitions
+table windowTestData
+|> window w1 as (partition by cate), w2 as (partition by val order by cate)
+|> select sum(val) over w1, first_value(cate) over w2;
+
+-- WINDOW with RANGE BETWEEN as part of the window definition
Review Comment:
silly style nitpick: can we add a period to the end of each of these
comments just to be consistent with the other similar comments in this file :)
##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala:
##########
@@ -5874,53 +5886,72 @@ class AstBuilder extends DataTypeAstBuilder
if (!SQLConf.get.getConf(SQLConf.OPERATOR_PIPE_SYNTAX_ENABLED)) {
operationNotAllowed("Operator pipe SQL syntax using |>", ctx)
}
- // This helper function adds a table subquery boundary between the new
operator to be added
- // (such as a filter or sort) and the input plan if one does not already
exist. This helps the
- // analyzer behave as if we had added the corresponding SQL clause after a
table subquery
- // containing the input plan.
- def withSubqueryAlias(): LogicalPlan = left match {
- case s: SubqueryAlias =>
- s
- case u: UnresolvedRelation =>
- u
- case _ =>
- SubqueryAlias(SubqueryAlias.generateSubqueryName(), left)
- }
- 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)
- }.getOrElse(Option(ctx.whereClause).map { c =>
- withWhereClause(c, withSubqueryAlias())
- }.getOrElse(Option(ctx.pivotClause()).map { c =>
- if (ctx.unpivotClause() != null) {
- throw
QueryParsingErrors.unpivotWithPivotInFromClauseNotAllowedError(ctx)
- }
- withPivot(c, left)
- }.getOrElse(Option(ctx.unpivotClause()).map { c =>
- if (ctx.pivotClause() != null) {
- throw
QueryParsingErrors.unpivotWithPivotInFromClauseNotAllowedError(ctx)
+
+ // Extract the base child and a list of WithWindowDefinition nodes from
the plan
Review Comment:
```suggestion
// If the subplan arriving as input for the pipe operator to apply on has
// any WINDOW clause, extract the window definition from the plan.
// We will apply it at the end.
```
##########
sql/core/src/test/resources/sql-tests/inputs/pipe-operators.sql:
##########
@@ -665,15 +665,119 @@ table t
table t
|> order by x sort by x;
--- The WINDOW clause is not supported yet.
+-- The WINDOW clause: positive tests.
+-------------------------------------
+-- WINDOW with one definition
table windowTestData
-|> window w as (partition by cte order by val)
+|> window w as (partition by cate order by val)
|> select cate, sum(val) over w;
+-- WINDOW with multiple definitions
+table windowTestData
+|> window w1 as (partition by cate), w2 as (partition by val order by cate)
+|> select sum(val) over w1, first_value(cate) over w2;
+
+-- WINDOW with RANGE BETWEEN as part of the window definition
+table windowTestData
+|> window w as (order by val_timestamp range between unbounded preceding and
current row)
+|> select avg(val) over w;
+
+-- WINDOW definition not being referred in the following SELECT clause
+table windowTestData
+|> window w as (partition by cate order by val)
+|> select cate;
+
+-- WINDOW definition not being referred in the following WHERE clause
+table windowTestData
+|> window w as (partition by cate order by val)
+|> where cate = "a";
+
+-- WINDOW definition not being referred in the following ORDER BY clause
+table windowTestData
+|> window w as (partition by cate order by val)
+|> order by cate;
+
+-- WINDOW referred in the following SELECT clause, with a WHERE after it
+table windowTestData
+|> window w as (partition by cate order by val)
+|> select cate, sum(val) over w
+|> where cate = "a";
+
+-- WINDOW referred in the following SELECT clause, with an ORDER BY after it
+table windowTestData
+|> window w as (partition by cate order by val)
+|> select cate, sum(val) over w as sum_val
+|> order by sum_val;
+
+-- WINDOW with table aliases to refer columns
+table windowTestData
+|> window w as (partition by windowTestData.cate order by windowTestData.val)
+|> select windowTestData.cate, sum(windowTestData.val) over w;
+
+-- WINDOW using struct fields.
+(select col from st)
+|> window w as (partition by col.i1 order by col.i2)
+|> select col.i1, sum(col.i2) over w;
+
+table st
+|> window w as (partition by st.col.i1 order by st.col.i2)
+|> select st.col.i1, sum(st.col.i2) over w;
+
+table st
+|> window w as (partition by spark_catalog.default.st.col.i1 order by
spark_catalog.default.st.col.i2)
+|> select spark_catalog.default.st.col.i1,
sum(spark_catalog.default.st.col.i2) over w;
+
+-- The WINDOW clause: debatable positive cases.
+------------------------------------------------
+-- WINDOW with nothing after it.
+table windowTestData
+|> window w as (partition by cate);
+
+table windowTestData
+|> window w as (partition by cate) order by val;
+
+-- WINDOW names can shadow other valid column names (align with the current
SQL syntax).
+table windowTestData
+|> window val as (partition by cate order by val)
+|> select cate, sum(val) over val;
+
+-- The WINDOW clause: negative tests.
+-------------------------------------
-- WINDOW and LIMIT are not supported at the same time.
table windowTestData
|> window w as (partition by cate order by val) limit 5;
+-- WINDOW and OFFSET are not supported at the same time.
+table windowTestData
+|> window w as (partition by cate order by val) offset 1;
+
+-- WINDOW and ORDER BY are not supported at the same time (in parallel).
+table windowTestData
+|> window w as (partition by cate) order by val
+|> select val, sum(val) over w;
+
+-- WINDOW and ORDER BY are not supported at the same time (in parallel)
+-- even if the WINDOW definition is not used
+table windowTestData
+|> window w as (partition by cate) order by val
+|> select val;
+
+-- WINDOW function is not supported in the WHERE clause.
+table windowTestData
+|> window w as (partition by cate) order by val
Review Comment:
this one is triggering a PARSE_SYNTAX_ERROR over the `order by val` part,
should we just remove that part in this particular test to exercise the error
behavior for the `|> where` clause following `|> window`?
##########
sql/core/src/test/resources/sql-tests/inputs/pipe-operators.sql:
##########
@@ -665,15 +665,119 @@ table t
table t
|> order by x sort by x;
--- The WINDOW clause is not supported yet.
+-- The WINDOW clause: positive tests.
+-------------------------------------
+-- WINDOW with one definition
table windowTestData
-|> window w as (partition by cte order by val)
+|> window w as (partition by cate order by val)
|> select cate, sum(val) over w;
+-- WINDOW with multiple definitions
+table windowTestData
+|> window w1 as (partition by cate), w2 as (partition by val order by cate)
+|> select sum(val) over w1, first_value(cate) over w2;
+
+-- WINDOW with RANGE BETWEEN as part of the window definition
+table windowTestData
+|> window w as (order by val_timestamp range between unbounded preceding and
current row)
+|> select avg(val) over w;
+
+-- WINDOW definition not being referred in the following SELECT clause
+table windowTestData
+|> window w as (partition by cate order by val)
+|> select cate;
+
+-- WINDOW definition not being referred in the following WHERE clause
+table windowTestData
+|> window w as (partition by cate order by val)
+|> where cate = "a";
+
+-- WINDOW definition not being referred in the following ORDER BY clause
+table windowTestData
+|> window w as (partition by cate order by val)
+|> order by cate;
+
+-- WINDOW referred in the following SELECT clause, with a WHERE after it
+table windowTestData
+|> window w as (partition by cate order by val)
+|> select cate, sum(val) over w
+|> where cate = "a";
+
+-- WINDOW referred in the following SELECT clause, with an ORDER BY after it
+table windowTestData
+|> window w as (partition by cate order by val)
+|> select cate, sum(val) over w as sum_val
+|> order by sum_val;
+
+-- WINDOW with table aliases to refer columns
+table windowTestData
+|> window w as (partition by windowTestData.cate order by windowTestData.val)
+|> select windowTestData.cate, sum(windowTestData.val) over w;
+
+-- WINDOW using struct fields.
+(select col from st)
+|> window w as (partition by col.i1 order by col.i2)
+|> select col.i1, sum(col.i2) over w;
+
+table st
+|> window w as (partition by st.col.i1 order by st.col.i2)
+|> select st.col.i1, sum(st.col.i2) over w;
+
+table st
+|> window w as (partition by spark_catalog.default.st.col.i1 order by
spark_catalog.default.st.col.i2)
+|> select spark_catalog.default.st.col.i1,
sum(spark_catalog.default.st.col.i2) over w;
+
+-- The WINDOW clause: debatable positive cases.
+------------------------------------------------
+-- WINDOW with nothing after it.
+table windowTestData
+|> window w as (partition by cate);
+
+table windowTestData
+|> window w as (partition by cate) order by val;
+
+-- WINDOW names can shadow other valid column names (align with the current
SQL syntax).
+table windowTestData
+|> window val as (partition by cate order by val)
+|> select cate, sum(val) over val;
+
+-- The WINDOW clause: negative tests.
+-------------------------------------
-- WINDOW and LIMIT are not supported at the same time.
table windowTestData
|> window w as (partition by cate order by val) limit 5;
+-- WINDOW and OFFSET are not supported at the same time.
+table windowTestData
+|> window w as (partition by cate order by val) offset 1;
+
+-- WINDOW and ORDER BY are not supported at the same time (in parallel).
+table windowTestData
+|> window w as (partition by cate) order by val
+|> select val, sum(val) over w;
+
+-- WINDOW and ORDER BY are not supported at the same time (in parallel)
+-- even if the WINDOW definition is not used
+table windowTestData
+|> window w as (partition by cate) order by val
+|> select val;
+
+-- WINDOW function is not supported in the WHERE clause.
+table windowTestData
+|> window w as (partition by cate) order by val
+|> where sum(val) over w = 1;
Review Comment:
we could also test some other SQL pipe operators besides `|> select` after
the `|> window` definition, to make sure they are rejected with reasonable
error messages. We now support `|> order by`, `|> aggregate` (if you sync from
the `master` branch), etc.
##########
sql/core/src/test/resources/sql-tests/inputs/pipe-operators.sql:
##########
@@ -665,15 +665,28 @@ table t
table t
|> order by x sort by x;
--- The WINDOW clause is not supported yet.
+-- The WINDOW clause: positive tests.
+-------------------------------------
table windowTestData
-|> window w as (partition by cte order by val)
+|> window w as (partition by cate order by val)
|> select cate, sum(val) over w;
+table windowTestData
+|> window w1 as (partition by cate), w2 as (partition by val order by cate)
+|> select sum(val) over w1, first_value(cate) over w2;
Review Comment:
If it's allowed in regular SQL, we should probably allow it the same way
here for consistency. It looks like we are exercising that in tests now and it
works.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]