You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@beam.apache.org by "Kirill Kozlov (Jira)" <ji...@apache.org> on 2019/11/13 01:03:00 UTC

[jira] [Commented] (BEAM-8042) Parsing of aggregate query fails

    [ https://issues.apache.org/jira/browse/BEAM-8042?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16972928#comment-16972928 ] 

Kirill Kozlov commented on BEAM-8042:
-------------------------------------

I tried running a similar query, but without COUNT(*) and it seems construct BeamRelNode successfully. 
{code:java}
ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);

String sql = "SELECT \n" +
    "  id, \n" +
    "  SUM(has_f1) as f1_count, \n" +
    "  SUM(has_f2) as f2_count, \n" +
    "  SUM(has_f3) as f3_count, \n" +
    "  SUM(has_f4) as f4_count, \n" +
    "  SUM(has_f5) as f5_count, \n" +
    "  SUM(has_f6) as f6_count  \n" +
    "FROM (select 0 as id, 0 as has_f1, 0 as has_f2, 0 as has_f3, 0 as has_f4, 0 as has_f5, 0 as has_f6)\n" +
    "GROUP BY id";

BeamRelNode node = zetaSQLQueryPlanner.convertToBeamRel(sql);
{code}
 

https://jira.apache.org/jira/browse/BEAM-7609 may also be relevant to AggregateProjectMergeRule.

> Parsing of aggregate query fails
> --------------------------------
>
>                 Key: BEAM-8042
>                 URL: https://issues.apache.org/jira/browse/BEAM-8042
>             Project: Beam
>          Issue Type: Sub-task
>          Components: dsl-sql-zetasql
>            Reporter: Rui Wang
>            Priority: Critical
>
> {code}
>   @Rule
>   public TestPipeline pipeline = TestPipeline.fromOptions(createPipelineOptions());
>   private static PipelineOptions createPipelineOptions() {
>     BeamSqlPipelineOptions opts = PipelineOptionsFactory.create().as(BeamSqlPipelineOptions.class);
>     opts.setPlannerName(ZetaSQLQueryPlanner.class.getName());
>     return opts;
>   }
>   @Test
>   public void testAggregate() {
>     Schema inputSchema = Schema.builder()
>         .addByteArrayField("id")
>         .addInt64Field("has_f1")
>         .addInt64Field("has_f2")
>         .addInt64Field("has_f3")
>         .addInt64Field("has_f4")
>         .addInt64Field("has_f5")
>         .addInt64Field("has_f6")
>         .build();
>     String sql = "SELECT \n" +
>         "  id, \n" +
>         "  COUNT(*) as count, \n" +
>         "  SUM(has_f1) as f1_count, \n" +
>         "  SUM(has_f2) as f2_count, \n" +
>         "  SUM(has_f3) as f3_count, \n" +
>         "  SUM(has_f4) as f4_count, \n" +
>         "  SUM(has_f5) as f5_count, \n" +
>         "  SUM(has_f6) as f6_count  \n" +
>         "FROM PCOLLECTION \n" +
>         "GROUP BY id";
>     pipeline
>         .apply(Create.empty(inputSchema))
>         .apply(SqlTransform.query(sql));
>     pipeline.run();
>   }
> {code}
> {code}
> Caused by: java.lang.RuntimeException: Error while applying rule AggregateProjectMergeRule, args [rel#553:LogicalAggregate.NONE(input=RelSubset#552,group={0},f1=COUNT(),f2=SUM($2),f3=SUM($3),f4=SUM($4),f5=SUM($5),f6=SUM($6),f7=SUM($7)), rel#551:LogicalProject.NONE(input=RelSubset#550,key=$0,f1=$1,f2=$2,f3=$3,f4=$4,f5=$5,f6=$6)]
> 	at org.apache.beam.repackaged.sql.org.apache.calcite.plan.volcano.VolcanoRuleCall.onMatch(VolcanoRuleCall.java:232)
> 	at org.apache.beam.repackaged.sql.org.apache.calcite.plan.volcano.VolcanoPlanner.findBestExp(VolcanoPlanner.java:637)
> 	at org.apache.beam.repackaged.sql.org.apache.calcite.tools.Programs$RuleSetProgram.run(Programs.java:340)
> 	at org.apache.beam.sdk.extensions.sql.zetasql.ZetaSQLPlannerImpl.transform(ZetaSQLPlannerImpl.java:168)
> 	at org.apache.beam.sdk.extensions.sql.zetasql.ZetaSQLQueryPlanner.parseQuery(ZetaSQLQueryPlanner.java:99)
> 	at org.apache.beam.sdk.extensions.sql.zetasql.ZetaSQLQueryPlanner.parseQuery(ZetaSQLQueryPlanner.java:87)
> 	at org.apache.beam.sdk.extensions.sql.zetasql.ZetaSQLQueryPlanner.convertToBeamRel(ZetaSQLQueryPlanner.java:66)
> 	at org.apache.beam.sdk.extensions.sql.impl.BeamSqlEnv.parseQuery(BeamSqlEnv.java:104)
> 	at 
> 	... 39 more
> Caused by: java.lang.ArrayIndexOutOfBoundsException: 7
> 	at org.apache.beam.repackaged.sql.com.google.common.collect.RegularImmutableList.get(RegularImmutableList.java:58)
> 	at org.apache.beam.repackaged.sql.org.apache.calcite.rel.rules.AggregateProjectMergeRule.apply(AggregateProjectMergeRule.java:96)
> 	at org.apache.beam.repackaged.sql.org.apache.calcite.rel.rules.AggregateProjectMergeRule.onMatch(AggregateProjectMergeRule.java:73)
> 	at org.apache.beam.repackaged.sql.org.apache.calcite.plan.volcano.VolcanoRuleCall.onMatch(VolcanoRuleCall.java:205)
> 	... 48 more
> {code}



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