You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@beam.apache.org by "David McIntosh (JIRA)" <ji...@apache.org> on 2019/04/03 16:48:00 UTC

[jira] [Created] (BEAM-6995) SQL aggregation with where clause fails to plan

David McIntosh created BEAM-6995:
------------------------------------

             Summary: SQL aggregation with where clause fails to plan
                 Key: BEAM-6995
                 URL: https://issues.apache.org/jira/browse/BEAM-6995
             Project: Beam
          Issue Type: Bug
          Components: dsl-sql
    Affects Versions: 2.11.0
            Reporter: David McIntosh


I'm finding that this code fails with a CannotPlanException listed below.
{code:java}
Schema schema = Schema.builder()
    .addInt32Field("id")
    .addInt32Field("val")
    .build();

Row row = Row.withSchema(schema).addValues(1, 2).build();

PCollection<Row> inputData = p.apply("row input", Create.of(row).withRowSchema(schema));

inputData.apply("sql",
    SqlTransform.query(
        "SELECT id, SUM(val) "
        + "FROM PCOLLECTION "
        + "WHERE val > 0 "
        + "GROUP BY id"));{code}
If the WHERE clause is removed the code runs successfully.

This may be similar to BEAM-5384 since I was able to work around this by adding an extra column to the input that isn't reference in the sql.
{code:java}
Schema schema = Schema.builder()
    .addInt32Field("id")
    .addInt32Field("val")
    .addInt32Field("extra")
    .build();{code}
 
{code:java}
org.apache.beam.repackaged.beam_sdks_java_extensions_sql.org.apache.calcite.plan.RelOptPlanner$CannotPlanException: Node [rel#100:Subset#2.BEAM_LOGICAL] could not be implemented; planner state:

Root: rel#100:Subset#2.BEAM_LOGICAL
Original rel:
LogicalAggregate(subset=[rel#100:Subset#2.BEAM_LOGICAL], group=[{0}], EXPR$1=[SUM($1)]): rowcount = 5.0, cumulative cost = {5.687500238418579 rows, 0.0 cpu, 0.0 io}, id = 98
  LogicalFilter(subset=[rel#97:Subset#1.NONE], condition=[>($1, 0)]): rowcount = 50.0, cumulative cost = {50.0 rows, 100.0 cpu, 0.0 io}, id = 96
    BeamIOSourceRel(subset=[rel#95:Subset#0.BEAM_LOGICAL], table=[[beam, PCOLLECTION]]): rowcount = 100.0, cumulative cost = {100.0 rows, 101.0 cpu, 0.0 io}, id = 92

Sets:
Set#0, type: RecordType(INTEGER id, INTEGER val)
        rel#95:Subset#0.BEAM_LOGICAL, best=rel#92, importance=0.7290000000000001
                rel#92:BeamIOSourceRel.BEAM_LOGICAL(table=[beam, PCOLLECTION]), rowcount=100.0, cumulative cost={100.0 rows, 101.0 cpu, 0.0 io}
        rel#110:Subset#0.ENUMERABLE, best=rel#109, importance=0.36450000000000005
                rel#109:BeamEnumerableConverter.ENUMERABLE(input=rel#95:Subset#0.BEAM_LOGICAL), rowcount=100.0, cumulative cost={1.7976931348623157E308 rows, 1.7976931348623157E308 cpu, 1.7976931348623157E308 io}
Set#1, type: RecordType(INTEGER id, INTEGER val)
        rel#97:Subset#1.NONE, best=null, importance=0.81
                rel#96:LogicalFilter.NONE(input=rel#95:Subset#0.BEAM_LOGICAL,condition=>($1, 0)), rowcount=50.0, cumulative cost={inf}
                rel#102:LogicalCalc.NONE(input=rel#95:Subset#0.BEAM_LOGICAL,expr#0..1={inputs},expr#2=0,expr#3=>($t1, $t2),id=$t0,val=$t1,$condition=$t3), rowcount=50.0, cumulative cost={inf}
        rel#104:Subset#1.BEAM_LOGICAL, best=rel#103, importance=0.405
                rel#103:BeamCalcRel.BEAM_LOGICAL(input=rel#95:Subset#0.BEAM_LOGICAL,expr#0..1={inputs},expr#2=0,expr#3=>($t1, $t2),id=$t0,val=$t1,$condition=$t3), rowcount=50.0, cumulative cost={150.0 rows, 801.0 cpu, 0.0 io}
        rel#106:Subset#1.ENUMERABLE, best=rel#105, importance=0.405
                rel#105:BeamEnumerableConverter.ENUMERABLE(input=rel#104:Subset#1.BEAM_LOGICAL), rowcount=50.0, cumulative cost={1.7976931348623157E308 rows, 1.7976931348623157E308 cpu, 1.7976931348623157E308 io}
Set#2, type: RecordType(INTEGER id, INTEGER EXPR$1)
        rel#99:Subset#2.NONE, best=null, importance=0.9
                rel#98:LogicalAggregate.NONE(input=rel#97:Subset#1.NONE,group={0},EXPR$1=SUM($1)), rowcount=5.0, cumulative cost={inf}
        rel#100:Subset#2.BEAM_LOGICAL, best=null, importance=1.0
                rel#101:AbstractConverter.BEAM_LOGICAL(input=rel#99:Subset#2.NONE,convention=BEAM_LOGICAL), rowcount=5.0, cumulative cost={inf}


        at org.apache.beam.repackaged.beam_sdks_java_extensions_sql.org.apache.calcite.plan.volcano.RelSubset$CheapestPlanReplacer.visit(RelSubset.java:437)
        at org.apache.beam.repackaged.beam_sdks_java_extensions_sql.org.apache.calcite.plan.volcano.RelSubset.buildCheapestPlan(RelSubset.java:296)
        at org.apache.beam.repackaged.beam_sdks_java_extensions_sql.org.apache.calcite.plan.volcano.VolcanoPlanner.findBestExp(VolcanoPlanner.java:657)
        at org.apache.beam.repackaged.beam_sdks_java_extensions_sql.org.apache.calcite.tools.Programs$RuleSetProgram.run(Programs.java:339)
        at org.apache.beam.repackaged.beam_sdks_java_extensions_sql.org.apache.calcite.prepare.PlannerImpl.transform(PlannerImpl.java:349)
        at org.apache.beam.sdk.extensions.sql.impl.BeamQueryPlanner.convertToBeamRel(BeamQueryPlanner.java:138)
        at org.apache.beam.sdk.extensions.sql.impl.BeamSqlEnv.parseQuery(BeamSqlEnv.java:143)
        at org.apache.beam.sdk.extensions.sql.SqlTransform.expand(SqlTransform.java:111)
        at org.apache.beam.sdk.extensions.sql.SqlTransform.expand(SqlTransform.java:79)
        at org.apache.beam.sdk.Pipeline.applyInternal(Pipeline.java:537)
        at org.apache.beam.sdk.Pipeline.applyTransform(Pipeline.java:488)
        at org.apache.beam.sdk.values.PCollection.apply(PCollection.java:370)
{code}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)