Skip to content

Conversation

Angryrou
Copy link
Contributor

@Angryrou Angryrou commented Oct 25, 2024

What changes were proposed in this pull request?

This PR adds SQL pipe syntax support for the WINDOW clause within the pipe SELECT operator |> SELECT.

For example

CREATE TEMPORARY VIEW t 
AS SELECT * from VALUES
  (1, 'apple', 1),
  (2, 'banana', 2),
  (3, 'apple', 3),
  (4, 'banana', 4),
AS t(id, name, amount);

TABLE t
|> SELECT id, name, amount, SUM(amount) OVER w
   WINDOW w AS (PARTITION BY name ORDER BY id);

1, apple, 1, 3
3, apple, 2, 3
2, banana, 3, 7
4, banana, 4, 7

Notes:

  1. |> WHERE is not extended to use the WINDOW clause because |> WHERE does not support window functions in its expressions.

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?

Yes

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

No

@github-actions github-actions bot added the SQL label Oct 25, 2024
@Angryrou Angryrou closed this Oct 25, 2024
@Angryrou Angryrou reopened this Oct 26, 2024
@Angryrou Angryrou marked this pull request as ready for review October 26, 2024 05:04
@dtenedor
Copy link
Contributor

cc @cloud-fan @gengliangwang :)

Copy link
Contributor

@dtenedor dtenedor left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for working on this!! I tried to think of testing ideas.

@Angryrou
Copy link
Contributor Author

Thank you @dtenedor for the consideration suggestions! I have updated the code accordingly and added more test cases.

@Angryrou
Copy link
Contributor Author

kindly ping @dtenedor

Copy link
Contributor

@dtenedor dtenedor left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The extra test coverage helps a lot, thanks for adding it!
(Apologies for delay in review, will try to respond faster next time.)

@Angryrou Angryrou force-pushed the pipe-window branch 2 times, most recently from 48e4332 to 7aae504 Compare November 1, 2024 21:37
@Angryrou Angryrou requested a review from dtenedor November 2, 2024 04:03
Copy link
Contributor

@dtenedor dtenedor left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks, this approach looks right! Just a final suggestion for a better error message in one location and an update to the SQL golden file tests to make them deterministic, then this PR should be complete.

@Angryrou
Copy link
Contributor Author

Angryrou commented Nov 4, 2024

Thanks again for the review! @dtenedor

Please feel free to let me know any other comments!

table windowTestData
|> select cate, val, sum(val) over w1, first_value(cate) over w2
window w1 as (partition by cate)
window w2 as (order by val)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'm a bit confused, aren't w1 and w2 both defined?

Copy link
Contributor Author

@Angryrou Angryrou Nov 6, 2024

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Similar to the classic SQL, when multiple WINDOW clauses are presented, the SQL parser will only see the first window definition, and hence miss the second window definition w2.

image

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

hmm, is it a bug? @srielau

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

If it is, it seems like the same bug for both regular SQL syntax and pipe operators, and the behavior would work the same for both (now, and if we were to change it later).

|> select cate, sum(val) over val
window val as (partition by cate order by val);

-- WINDOW definition can be referred in the downstream SELECT clause.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Is this an expected behavior in the SQL pipe syntax spec? cc @dtenedor @srielau

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@Angryrou the query in this test case should fail now, since the window w is not yet defined in the |> select operator. Do we need to re-generate the golden files?

Copy link
Contributor Author

@Angryrou Angryrou Nov 8, 2024

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@cloud-fan @dtenedor

Fixed! I added logic to ensure that each window name used in UnresolvedWindowExpression is properly defined within the same |> SELECT clause.

In the previous logic, UnresolvedWindowExpression was analyzed only after scanning all pipe operators. As a result, in this test case, the window w in the first |> SELECT clause could incorrectly reference the definition in the second |> SELECT.

With this change, an error will be raised with INVALID_SQL_SYNTAX.WINDOW_REFERENCE_NOT_FOUND if a window name is used but no window clause exists, or INVALID_SQL_SYNTAX.UNRESOLVED_WINDOW_REFERENCE if a window name in UnresolvedWindowExpression is not defined within the same |> SELECT clause.

Copy link
Contributor

@dtenedor dtenedor left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Generally LGTM, just have a question about one remaining test case.

|> select cate, sum(val) over val
window val as (partition by cate order by val);

-- WINDOW definition can be referred in the downstream SELECT clause.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@Angryrou the query in this test case should fail now, since the window w is not yet defined in the |> select operator. Do we need to re-generate the golden files?

table windowTestData
|> select cate, val, sum(val) over w1, first_value(cate) over w2
window w1 as (partition by cate)
window w2 as (order by val)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

If it is, it seems like the same bug for both regular SQL syntax and pipe operators, and the behavior would work the same for both (now, and if we were to change it later).

throw QueryParsingErrors.cannotFindWindowReferenceError(unresolvedWindowNames.head, ctx)
} else {
// Find any unresolved window names not defined in windowDefs
unresolvedWindowNames.find(!windowDefs.contains(_)) match {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It's better to not do analysis work in the parser, as we may miss something when re-implementing the analysis work. E.g. shall we consider case sensitivity here?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@Angryrou can you highlight the logical plan differences between classic and pipe SQL syntaxes? I'd like to understand why we can reference previously defined window frame in the pipe syntax and see how we can fix it in the analyzer.

Copy link
Contributor Author

@Angryrou Angryrou Nov 11, 2024

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Differences @cloud-fan

  1. with the pipe syntax, the unresolvedwindowexpression is wrapped by a pipeselect expression
  2. the classic syntax has an auto-generated subquery name for the subquery

In the pipe syntax, the WINDOW clause in the 2nd |> SELECT leads to a WithWindowDefinition for w at the top of the plan tree. The 1st |> SELECT seems to be able to get the information as well.

In the classic syntax, the outer query defines w and gets a WithWindowDefinition at the top of the tree as well. So the inner subquery is able to get the window definition.

SQLs

-- pipe syntax
table windowTestData
|> select cate, val, first_value(cate) over w as first_val
|> select cate, val, sum(val) over w as sum_val
   window w as (order by val);

-- classic syntax
select cate, val, sum(val) over w as sum_val
from (
  select cate, val, first_value(cate) over w as first_val
  from windowTestData
)
window w as (order by val);

Parsed Logical Plans (parser.parsePlan(...))

-- from pipe syntax
'WithWindowDefinition [w=windowspecdefinition('val ASC NULLS FIRST, unspecifiedframe$())]
+- 'Project ['cate, 'val, pipeselect(unresolvedwindowexpression('sum('val), WindowSpecReference(w))) AS sum_val#3]
   +- 'Project ['cate, 'val, pipeselect(unresolvedwindowexpression('first_value('cate), WindowSpecReference(w))) AS first_val#2]
      +- 'UnresolvedRelation [windowTestData], [], false

-- from classic syntax
'WithWindowDefinition [w=windowspecdefinition('val ASC NULLS FIRST, unspecifiedframe$())]
+- 'Project ['cate, 'val, unresolvedwindowexpression('sum('val), WindowSpecReference(w)) AS sum_val#1]
   +- 'SubqueryAlias __auto_generated_subquery_name
      +- 'Project ['cate, 'val, unresolvedwindowexpression('first_value('cate), WindowSpecReference(w)) AS first_val#0]
         +- 'UnresolvedRelation [windowTestData], [], false

Analyzed Logical Plans (from the golden file)
https://docs.google.com/document/d/1qa1jUAoWa0aS5037oazydJRYQSAFFklUlPmnwLxOZDM/edit?usp=sharing

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@Angryrou thanks for the detailed explanation!

Looking at the analyzer rule WindowsSubstitution, we actually allow all the child nodes to reference the window frame definitions inside WithWindowDefinition. This makes sense for classic SQL, as the WINDOW clause is for the entire query, but it's pretty weird for the pipe SQL.

It looks like we want a variant of WithWindowDefinition that only works for its direct child node. How about we add a flag forPipeSQL in WithWindowDefinition, and only resolve the window frames in the direct child.

  object WindowsSubstitution extends Rule[LogicalPlan] {
    def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUpWithPruning(
      _.containsAnyPattern(WITH_WINDOW_DEFINITION, UNRESOLVED_WINDOW_EXPRESSION), ruleId) {
      // Lookup WindowSpecDefinitions. This rule works with unresolved children.
      case WithWindowDefinition(windowDefinitions, child) => child.resolveExpressions {
        case UnresolvedWindowExpression(c, WindowSpecReference(windowName)) =>
          val windowSpecDefinition = windowDefinitions.getOrElse(windowName,
            throw QueryCompilationErrors.windowSpecificationNotDefinedError(windowName))
          WindowExpression(c, windowSpecDefinition)
      }
    }
  }

child.resolveExpressions actually traverses down the entire plan tree, and we should call child.transformExpressions to only resolve expressions in the current node, when the forPipeSQL flag is true.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@cloud-fan Thanks for the guidance! I have changed it accordingly.

@Angryrou Angryrou requested a review from cloud-fan November 12, 2024 16:02
Copy link
Contributor

@dtenedor dtenedor left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This solution generally LGTM, if Wenchen agrees :)

@cloud-fan
Copy link
Contributor

thanks, merging to master!

@cloud-fan cloud-fan closed this in 82040bb Nov 13, 2024
@Angryrou Angryrou deleted the pipe-window branch November 18, 2024 00:28
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
Projects
None yet
Development

Successfully merging this pull request may close these issues.

3 participants