Andrew Pilloud created BEAM-12759:
-------------------------------------

             Summary: ORDER BY column not in SELECT crashes
                 Key: BEAM-12759
                 URL: https://issues.apache.org/jira/browse/BEAM-12759
             Project: Beam
          Issue Type: Bug
          Components: dsl-sql-zetasql
            Reporter: Andrew Pilloud
            Assignee: Andrew Pilloud


`SELECT Value FROM KeyValue ORDER BY Key DESC LIMIT 2` doesn't work.

This test should pass...
{code}
+  @Test
+  public void testZetaSQLSelectFromTableOrderByNoSelectLimit() {
+    String sql = "SELECT Value FROM KeyValue ORDER BY Key DESC LIMIT 2;";
+    PCollection<Row> stream = execute(sql);
+
+    final Schema schema = Schema.builder().addStringField("field2").build();
+    PAssert.that(stream)
+        .containsInAnyOrder(
+            Row.withSchema(schema).addValues("KeyValue234").build(),
+            Row.withSchema(schema).addValues("KeyValue235").build());
+
+    
pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES));
+  }
{code}

It crashes instead:
{code}
java.lang.IndexOutOfBoundsException: index (-1) must not be negative            
                                                           
        at 
org.apache.beam.vendor.calcite.v1_20_0.com.google.common.base.Preconditions.checkElementIndex(Preconditions.java:310)
           
        at 
org.apache.beam.vendor.calcite.v1_20_0.com.google.common.base.Preconditions.checkElementIndex(Preconditions.java:293)
           
        at 
org.apache.beam.vendor.calcite.v1_20_0.com.google.common.collect.SingletonImmutableList.get(SingletonImmutableList.java:41)
     
        at 
org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexBuilder.makeInputRef(RexBuilder.java:855)
                      
        at 
org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.Sort.<init>(Sort.java:103)
                                   
        at 
org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.logical.LogicalSort.<init>(LogicalSort.java:37)
                   
        at 
org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.logical.LogicalSort.create(LogicalSort.java:63)
                   
        at 
org.apache.beam.sdk.extensions.sql.zetasql.translation.LimitOffsetScanToOrderByLimitConverter.convert(LimitOffsetScanToOrderByLi
mitConverter.java:86)                                                           
                                                           
        at 
org.apache.beam.sdk.extensions.sql.zetasql.translation.LimitOffsetScanToOrderByLimitConverter.convert(LimitOffsetScanToOrderByLi
mitConverter.java:42)                                                           
                                                           
        at 
org.apache.beam.sdk.extensions.sql.zetasql.translation.QueryStatementConverter.convertNode(QueryStatementConverter.java:102)
    
        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:197)
           
        at 
org.apache.beam.sdk.extensions.sql.zetasql.ZetaSQLQueryPlanner.convertToBeamRel(ZetaSQLQueryPlanner.java:185)
                   
        at 
org.apache.beam.sdk.extensions.sql.zetasql.ZetaSqlDialectSpecTest.execute(ZetaSqlDialectSpecTest.java:74)
                       
        at 
org.apache.beam.sdk.extensions.sql.zetasql.ZetaSqlDialectSpecTest.execute(ZetaSqlDialectSpecTest.java:79)
                       
        at 
org.apache.beam.sdk.extensions.sql.zetasql.ZetaSqlDialectSpecTest.testZetaSQLSelectFromTableOrderByNoSelectLimit(ZetaSqlDialectS
pecTest.java:1297)
{code}



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to