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

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

                Author: ASF GitHub Bot
            Created on: 05/May/22 21:38
            Start Date: 05/May/22 21:38
    Worklog Time Spent: 10m 
      Work Description: apilloud commented on PR #16911:
URL: https://github.com/apache/beam/pull/16911#issuecomment-1119071169

   I printed node and fieldlist at the beginning of retrieveRexNode.
   ```
       Node: ProjectScan
       +-column_list=[$analytic.$analytic1#5]
       +-input_scan=
         +-AnalyticScan
           +-column_list=[KeyValue.Key#1, KeyValue.Value#2, KeyValue.ts#3, 
$analytic.$analytic1#5]
           +-input_scan=
           | +-TableScan(column_list=KeyValue.[Key#1, Value#2, ts#3], 
table=KeyValue, column_index_list=[0, 1, 2])
           +-function_group_list=
             +-AnalyticFunctionGroup
               +-analytic_function_list=
                 +-$analytic1#5 :=
                   +-AnalyticFunctionCall(ZetaSQL:sum(INT64) -> INT64)
                     +-ColumnRef(type=INT64, column=KeyValue.Key#1)
                     +-window_frame=
                       +-WindowFrame(frame_unit=ROWS)
                         +-start_expr=
                         | +-WindowFrameExpr(boundary_type=UNBOUNDED PRECEDING)
                         +-end_expr=
                           +-WindowFrameExpr(boundary_type=UNBOUNDED FOLLOWING)
   
       FieldList: [#0: $col1 BIGINT]
   ```
   
   If you walk up the stack one level, you'll see that this function call is 
`retrieveRexNode(zetaNode, input.getRowType().getFieldList())` This input does 
not match what we expect, the input FieldList has only 1 column but the ZetaSQL 
input AnalyticScan has 4 columns. That input is produced in 
`AnalyticScanConverter`, which is the code you are adding in this PR. It looks 
like the ZetaSQL `AnalyticScan` passes through all of the columns from the 
input but those are missing from your conversion. See the description of what 
it does here: 
https://github.com/google/zetasql/blob/1933acefe8eb10e293309a9fa7d7c7f1cb6bf8c4/zetasql/analyzer/analytic_function_resolver.cc#L870




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

    Worklog Id:     (was: 766940)
    Time Spent: 2h 50m  (was: 2h 40m)

> 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: 2h 50m
>  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