[ 
https://issues.apache.org/jira/browse/BEAM-12097?focusedWorklogId=763126&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-763126
 ]

ASF GitHub Bot logged work on BEAM-12097:
-----------------------------------------

                Author: ASF GitHub Bot
            Created on: 27/Apr/22 19:30
            Start Date: 27/Apr/22 19:30
    Worklog Time Spent: 10m 
      Work Description: roger-mike commented on PR #16911:
URL: https://github.com/apache/beam/pull/16911#issuecomment-1111399812

   Hi @apilloud, I asked the Calcite devs about this task, specifically about 
how to create a Relational Operator Tree for a window function.
   
   I changed the approach and added the window functions as ```RexOver``` to a 
```LogicalProject```, instead of building the tree from a ```LogicalWindow```.  
This way we build a simpler tree and let part of the conversion to the Calcite 
planner. This is the comment I received from Julian Hyde on that matter:
   
   > It’s complicated. Windowed aggregates start off as instances of RexOver 
expressions inside a Project and are later converted (by ProjectToWindowRule 
[1]) into Window relational operators.
   > 
   > Both of these representations exist in RelNode-land and I’m not sure which 
should be considered ‘canonical’. Window is a little bit more ‘physical’ than 
Project. At some stage in planning, it is assumed that all RexOver instances 
have been converted.
   > 
   >
   >  Julian
   > 
   > [1] 
https://github.com/apache/calcite/blob/bebe473fab2e242736614659ed6e5d04eeeb8bf5/core/src/main/java/org/apache/calcite/rel/rules/ProjectToWindowRule.java#L74
 
<https://github.com/apache/calcite/blob/bebe473fab2e242736614659ed6e5d04eeeb8bf5/core/src/main/java/org/apache/calcite/rel/rules/ProjectToWindowRule.java#L74>
   
   So far so good. I tested and it seems it creates a valid BEAMPlan. The issue 
here is that I had to hardcode the value of this [index 
]([beam/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/ExpressionConverter.java](https://github.com/apache/beam/blob/77e924fd5a025326b50352c29598ea1eb48d385a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/ExpressionConverter.java#L213))to
 make it point to a valid column reference. Not sure how to solve this issue. I 
was hoping you could give me some pointers on this. Thank you.
   
   P.S.
   I also got this one comment, I think the Calcite community might be 
interested in this translation feature. I said I was going to pass the message. 
What do you think?
   
   > It’s very interesting that you want to convert ZetaSQL to and from 
Calcite. I think other people will be interested in this. (It would help some 
of the things I am doing in my $dayjob at Google.) Would you consider 
contributing this as a new module in Calcite? Or creating a standalone project 
on GitHub?
   > 
   > Julian
   
   




Issue Time Tracking
-------------------

    Worklog Id:     (was: 763126)
    Time Spent: 1h 40m  (was: 1.5h)

> OVER clause converted from ZetaSQL to Logical SQL plan
> ------------------------------------------------------
>
>                 Key: BEAM-12097
>                 URL: https://issues.apache.org/jira/browse/BEAM-12097
>             Project: Beam
>          Issue Type: New Feature
>          Components: dsl-sql-zetasql
>            Reporter: Andrew Pilloud
>            Assignee: Mike Hernandez
>            Priority: P3
>          Time Spent: 1h 40m
>  Remaining Estimate: 0h
>
> The OVER clause isn't supported by our ZetaSQL to Calcite translator. It can 
> be trivially enabled in the parser with the example below, but there is some 
> work required to convert the parsed ZetaSQL proto to Calcite logical 
> operators (mostly in AggregateScanConverter).
> This is the "over clause" TODO here: 
> https://github.com/apache/beam/blob/243128a8fc52798e1b58b0cf1a271d95ee7aa241/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/AggregateScanConverter.java#L147
> {code:java}
> a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/SqlAnalyzer.java
> +++ 
> b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/SqlAnalyzer.java
> @@ -144,6 +144,7 @@ public class SqlAnalyzer {
>          .setEnabledLanguageFeatures(
>              new HashSet<>(
>                  Arrays.asList(
> +                    LanguageFeature.FEATURE_ANALYTIC_FUNCTIONS,
>                      LanguageFeature.FEATURE_NUMERIC_TYPE,
>                      LanguageFeature.FEATURE_DISALLOW_GROUP_BY_FLOAT,
>                      LanguageFeature.FEATURE_V_1_2_CIVIL_TIME,
> diff --git 
> a/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlDialectSpecTest.java
>  
> b/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlDialectSpecTest.java
> index 33889f34884..fd107ac5721 100644
> --- 
> a/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlDialectSpecTest.java
> +++ 
> b/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlDialectSpecTest.java
> @@ -3461,6 +3461,16 @@ public class ZetaSqlDialectSpecTest extends 
> ZetaSqlTestBase {
>      zetaSQLQueryPlanner.convertToBeamRel(sql);
>    }
>  
> +  @Test
> +  public void testAnalyticOver() {
> +    String sql = "select sum(Key) over () From KeyValue";
> +
> +    ZetaSQLQueryPlanner zetaSQLQueryPlanner = new 
> ZetaSQLQueryPlanner(config);
> +    thrown.expect(UnsupportedOperationException.class);
> +    thrown.expectMessage("Does not support sub-queries");
> +    zetaSQLQueryPlanner.convertToBeamRel(sql);
> +  }
> +
>    @Test
>    public void testSubstr() {
>      String sql = "SELECT substr(@p0, @p1, @p2)"; {code}
> Current state the test fails:
> {code}
> java.lang.UnsupportedOperationException: Conversion of RESOLVED_ANALYTIC_SCAN 
> is not supported                             
>         at 
> org.apache.beam.sdk.extensions.sql.zetasql.translation.QueryStatementConverter.getConverterRule(QueryStatementConverter.java:108
> )                                                                             
>                                                              
>         at 
> org.apache.beam.sdk.extensions.sql.zetasql.translation.QueryStatementConverter.convertNode(QueryStatementConverter.java:99)
>      
>         at 
> java.base/java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:195)
>                                              
>         at 
> java.base/java.util.Collections$2.tryAdvance(Collections.java:4756)           
>                                                   
>         at 
> java.base/java.util.Collections$2.forEachRemaining(Collections.java:4764)     
>                                                   
>         at 
> java.base/java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:484)
>                                                  
>         at 
> java.base/java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:474)
>                                           
>         at 
> java.base/java.util.stream.ReduceOps$ReduceOp.evaluateSequential(ReduceOps.java:913)
>                                             
>         at 
> java.base/java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:234)
>                                                  
>         at 
> java.base/java.util.stream.ReferencePipeline.collect(ReferencePipeline.java:578)
>                                                 
>         at 
> org.apache.beam.sdk.extensions.sql.zetasql.translation.QueryStatementConverter.convertNode(QueryStatementConverter.java:101)
>     
>         at 
> org.apache.beam.sdk.extensions.sql.zetasql.translation.QueryStatementConverter.convert(QueryStatementConverter.java:89)
>          
>         at 
> org.apache.beam.sdk.extensions.sql.zetasql.translation.QueryStatementConverter.convertRootQuery(QueryStatementConverter.java:55)
>         at 
> org.apache.beam.sdk.extensions.sql.zetasql.ZetaSQLPlannerImpl.rel(ZetaSQLPlannerImpl.java:98)
>                                    
>         at 
> org.apache.beam.sdk.extensions.sql.zetasql.ZetaSQLQueryPlanner.convertToBeamRelInternal(ZetaSQLQueryPlanner.java:313)
>            
>         at 
> org.apache.beam.sdk.extensions.sql.zetasql.ZetaSQLQueryPlanner.convertToBeamRel(ZetaSQLQueryPlanner.java:301)
>                    
>         at 
> org.apache.beam.sdk.extensions.sql.zetasql.ZetaSQLQueryPlanner.convertToBeamRel(ZetaSQLQueryPlanner.java:285)
>                    
>         at 
> org.apache.beam.sdk.extensions.sql.zetasql.ZetaSqlDialectSpecTest.testAnalyticOver(ZetaSqlDialectSpecTest.java:3471)
>    
> {code}



--
This message was sent by Atlassian Jira
(v8.20.7#820007)

Reply via email to