You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@beam.apache.org by "Andrew Pilloud (Jira)" <ji...@apache.org> on 2020/05/21 23:35:00 UTC

[jira] [Reopened] (BEAM-9514) AssertionError type mismatch from SUM

     [ https://issues.apache.org/jira/browse/BEAM-9514?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Andrew Pilloud reopened BEAM-9514:
----------------------------------

Still seeing this class of issue:

11 failures in shard 4, 2 failures in shard 8, 1 failure in shard 23, 4 failures in shard 30, 
{code}
SELECT COUNT(bool_val) FROM TableAllNull
BIGINT NOT NULL
	at org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.util.Litmus$1.fail(Litmus.java:31)
	at org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptUtil.eq(RelOptUtil.java:1958)
	at org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.Aggregate.typeMatchesInferred(Aggregate.java:434)
	at org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.Aggregate.<init>(Aggregate.java:159)
	at org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.logical.LogicalAggregate.<init>(LogicalAggregate.java:65)
	at org.apache.beam.sdk.extensions.sql.zetasql.translation.AggregateScanConverter.convert(AggregateScanConverter.java:110)
	at org.apache.beam.sdk.extensions.sql.zetasql.translation.AggregateScanConverter.convert(AggregateScanConverter.java:50)
	at org.apache.beam.sdk.extensions.sql.zetasql.translation.QueryStatementConverter.convertNode(QueryStatementConverter.java:98)
	at java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:193)
	at java.util.Collections$2.tryAdvance(Collections.java:4717)
	at java.util.Collections$2.forEachRemaining(Collections.java:4725)
	at java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:481)
	at java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:471)
	at java.util.stream.ReduceOps$ReduceOp.evaluateSequential(ReduceOps.java:708)
	at java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:234)
	at java.util.stream.ReferencePipeline.collect(ReferencePipeline.java:499)
	at org.apache.beam.sdk.extensions.sql.zetasql.translation.QueryStatementConverter.convertNode(QueryStatementConverter.java:97)
	at org.apache.beam.sdk.extensions.sql.zetasql.translation.QueryStatementConverter.convert(QueryStatementConverter.java:85)
	at org.apache.beam.sdk.extensions.sql.zetasql.translation.QueryStatementConverter.convertRootQuery(QueryStatementConverter.java:51)
	at org.apache.beam.sdk.extensions.sql.zetasql.ZetaSQLPlannerImpl.rel(ZetaSQLPlannerImpl.java:160)
	at org.apache.beam.sdk.extensions.sql.zetasql.ZetaSQLQueryPlanner.convertToBeamRelInternal(ZetaSQLQueryPlanner.java:131)
	at org.apache.beam.sdk.extensions.sql.zetasql.ZetaSQLQueryPlanner.convertToBeamRel(ZetaSQLQueryPlanner.java:115)
	at cloud.dataflow.sql.ExecuteQueryServiceServer$SqlComplianceServiceImpl.executeQuery(ExecuteQueryServiceServer.java:241)
	at com.google.zetasql.testing.SqlComplianceServiceGrpc$MethodHandlers.invoke(SqlComplianceServiceGrpc.java:423)
	at com.google.zetasql.io.grpc.stub.ServerCalls$UnaryServerCallHandler$UnaryServerCallListener.onHalfClose(ServerCalls.java:171)
	at com.google.zetasql.io.grpc.internal.ServerCallImpl$ServerStreamListenerImpl.halfClosed(ServerCallImpl.java:283)
	at com.google.zetasql.io.grpc.internal.ServerImpl$JumpToApplicationThreadServerStreamListener$1HalfClosed.runInContext(ServerImpl.java:711)
	at com.google.zetasql.io.grpc.internal.ContextRunnable.run(ContextRunnable.java:37)
	at com.google.zetasql.io.grpc.internal.SerializingExecutor.run(SerializingExecutor.java:123)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
	at java.lang.Thread.run(Thread.java:748)
{code}


> AssertionError type mismatch from SUM
> -------------------------------------
>
>                 Key: BEAM-9514
>                 URL: https://issues.apache.org/jira/browse/BEAM-9514
>             Project: Beam
>          Issue Type: Bug
>          Components: dsl-sql-zetasql
>            Reporter: Andrew Pilloud
>            Assignee: Andrew Pilloud
>            Priority: P1
>              Labels: zetasql-compliance
>             Fix For: 2.22.0
>
>          Time Spent: 20m
>  Remaining Estimate: 0h
>
> Six failures in shard 31
> {code}
> 	at org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.util.Litmus$1.fail(Litmus.java:31)
> 	at org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptUtil.equal(RelOptUtil.java:1984)
> 	at org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.volcano.RelSubset.add(RelSubset.java:284)
> 	at org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.volcano.RelSet.add(RelSet.java:148)
> 	at org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.volcano.VolcanoPlanner.addRelToSet(VolcanoPlanner.java:1806)
> 	at org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.volcano.VolcanoPlanner.reregister(VolcanoPlanner.java:1480)
> 	at org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.volcano.RelSet.mergeWith(RelSet.java:331)
> 	at org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.volcano.VolcanoPlanner.merge(VolcanoPlanner.java:1571)
> 	at org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.volcano.VolcanoPlanner.ensureRegistered(VolcanoPlanner.java:863)
> 	at org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.volcano.VolcanoPlanner.ensureRegistered(VolcanoPlanner.java:1927)
> 	at org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.volcano.VolcanoRuleCall.transformTo(VolcanoRuleCall.java:129)
> 	at org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptRuleCall.transformTo(RelOptRuleCall.java:236)
> 	at org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.rules.AggregateRemoveRule.onMatch(AggregateRemoveRule.java:126)
> 	at org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.volcano.VolcanoRuleCall.onMatch(VolcanoRuleCall.java:208)
> 	at org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.volcano.VolcanoPlanner.findBestExp(VolcanoPlanner.java:631)
> 	at org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.tools.Programs$RuleSetProgram.run(Programs.java:328)
> 	at org.apache.beam.sdk.extensions.sql.zetasql.ZetaSQLPlannerImpl.transform(ZetaSQLPlannerImpl.java:180)
> 	at org.apache.beam.sdk.extensions.sql.zetasql.ZetaSQLQueryPlanner.convertToBeamRelInternal(ZetaSQLQueryPlanner.java:150)
> 	at org.apache.beam.sdk.extensions.sql.zetasql.ZetaSQLQueryPlanner.convertToBeamRel(ZetaSQLQueryPlanner.java:115)
> 	at cloud.dataflow.sql.ExecuteQueryServiceServer$SqlComplianceServiceImpl.executeQuery(ExecuteQueryServiceServer.java:242)
> 	at com.google.zetasql.testing.SqlComplianceServiceGrpc$MethodHandlers.invoke(SqlComplianceServiceGrpc.java:423)
> 	at com.google.zetasql.io.grpc.stub.ServerCalls$UnaryServerCallHandler$UnaryServerCallListener.onHalfClose(ServerCalls.java:171)
> 	at com.google.zetasql.io.grpc.internal.ServerCallImpl$ServerStreamListenerImpl.halfClosed(ServerCallImpl.java:283)
> 	at com.google.zetasql.io.grpc.internal.ServerImpl$JumpToApplicationThreadServerStreamListener$1HalfClosed.runInContext(ServerImpl.java:711)
> 	at com.google.zetasql.io.grpc.internal.ContextRunnable.run(ContextRunnable.java:37)
> 	at com.google.zetasql.io.grpc.internal.SerializingExecutor.run(SerializingExecutor.java:123)
> 	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
> 	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
> 	at java.lang.Thread.run(Thread.java:748)
> {code}
> 1:
> {code}
> Apr 01, 2020 11:48:56 AM cloud.dataflow.sql.ExecuteQueryServiceServer$SqlComplianceServiceImpl executeQuery
> INFO: Processing Sql statement: select sum(distinct_4) from TableDistincts
> group by distinct_2
> having false
> Apr 01, 2020 11:48:57 AM cloud.dataflow.sql.ExecuteQueryServiceServer$SqlComplianceServiceImpl executeQuery
> SEVERE: !!!! Type mismatch:
> rowtype of new rel:
> RecordType(BIGINT distinct_2, BIGINT $col1) NOT NULL
> rowtype of set:
> RecordType(BIGINT distinct_2, BIGINT NOT NULL $col1) NOT NULL
> java.lang.AssertionError: Type mismatch:
> rowtype of new rel:
> RecordType(BIGINT distinct_2, BIGINT $col1) NOT NULL
> rowtype of set:
> RecordType(BIGINT distinct_2, BIGINT NOT NULL $col1) NOT NULL
> 	at org.apache.beam.vendor.calcite.v1_20_0.org.apache.cal
> {code}
> 2:
> {code}
> INFO: Processing Sql statement: select sum(distinct_4) from TableDistincts
> group by distinct_2
> having sum(distinct_4) > 14 and false
> Apr 01, 2020 11:48:58 AM cloud.dataflow.sql.ExecuteQueryServiceServer$SqlComplianceServiceImpl executeQuery
> SEVERE: !!!! Type mismatch:
> rowtype of new rel:
> RecordType(BIGINT distinct_2, BIGINT $col1, BIGINT $agg2) NOT NULL
> rowtype of set:
> RecordType(BIGINT distinct_2, BIGINT NOT NULL $col1, BIGINT NOT NULL $agg2) NOT NULL
> java.lang.AssertionError: Type mismatch:
> rowtype of new rel:
> RecordType(BIGINT distinct_2, BIGINT $col1, BIGINT $agg2) NOT NULL
> rowtype of set:
> RecordType(BIGINT distinct_2, BIGINT NOT NULL $col1, BIGINT NOT NULL $agg2) NOT NULL
> {code}
> 3:
> {code}
> Apr 01, 2020 11:48:58 AM cloud.dataflow.sql.ExecuteQueryServiceServer$SqlComplianceServiceImpl executeQuery
> INFO: Processing Sql statement: select sum(distinct_4) from TableDistincts
> group by distinct_2
> having sum(distinct_4) > 14 and null
> Apr 01, 2020 11:48:58 AM cloud.dataflow.sql.ExecuteQueryServiceServer$SqlComplianceServiceImpl executeQuery
> SEVERE: !!!! Type mismatch:
> rowtype of new rel:
> RecordType(BIGINT distinct_2, BIGINT $col1, BIGINT $agg2) NOT NULL
> rowtype of set:
> RecordType(BIGINT distinct_2, BIGINT NOT NULL $col1, BIGINT NOT NULL $agg2) NOT NULL
> java.lang.AssertionError: Type mismatch:
> rowtype of new rel:
> RecordType(BIGINT distinct_2, BIGINT $col1, BIGINT $agg2) NOT NULL
> rowtype of set:
> RecordType(BIGINT distinct_2, BIGINT NOT NULL $col1, BIGINT NOT NULL $agg2) NOT NULL
> {code}
> 4:
> {code}
> Apr 01, 2020 11:49:04 AM cloud.dataflow.sql.ExecuteQueryServiceServer$SqlComplianceServiceImpl executeQuery
> INFO: Processing Sql statement: select sum(distinct_4) from TableDistinctsWithNulls
> group by distinct_2
> having false
> Apr 01, 2020 11:49:04 AM cloud.dataflow.sql.ExecuteQueryServiceServer$SqlComplianceServiceImpl executeQuery
> SEVERE: !!!! Type mismatch:
> rowtype of new rel:
> RecordType(BIGINT distinct_2, BIGINT $col1) NOT NULL
> rowtype of set:
> RecordType(BIGINT distinct_2, BIGINT NOT NULL $col1) NOT NULL
> java.lang.AssertionError: Type mismatch:
> rowtype of new rel:
> RecordType(BIGINT distinct_2, BIGINT $col1) NOT NULL
> rowtype of set:
> RecordType(BIGINT distinct_2, BIGINT NOT NULL $col1) NOT NULL
> {code}
> 5:
> {code}
> Apr 01, 2020 11:49:04 AM cloud.dataflow.sql.ExecuteQueryServiceServer$SqlComplianceServiceImpl executeQuery
> INFO: Processing Sql statement: select sum(distinct_4) from TableDistinctsWithNulls
> group by distinct_2
> having sum(distinct_4) > 14 and false
> Apr 01, 2020 11:49:05 AM cloud.dataflow.sql.ExecuteQueryServiceServer$SqlComplianceServiceImpl executeQuery
> SEVERE: !!!! Type mismatch:
> rowtype of new rel:
> RecordType(BIGINT distinct_2, BIGINT $col1, BIGINT $agg2) NOT NULL
> rowtype of set:
> RecordType(BIGINT distinct_2, BIGINT NOT NULL $col1, BIGINT NOT NULL $agg2) NOT NULL
> java.lang.AssertionError: Type mismatch:
> rowtype of new rel:
> RecordType(BIGINT distinct_2, BIGINT $col1, BIGINT $agg2) NOT NULL
> rowtype of set:
> RecordType(BIGINT distinct_2, BIGINT NOT NULL $col1, BIGINT NOT NULL $agg2) NOT NULL
> {code}
> 6:
> {code}
> Apr 01, 2020 11:49:05 AM cloud.dataflow.sql.ExecuteQueryServiceServer$SqlComplianceServiceImpl executeQuery
> INFO: Processing Sql statement: select sum(distinct_4) from TableDistinctsWithNulls
> group by distinct_2
> having sum(distinct_4) > 14 and null
> Apr 01, 2020 11:49:05 AM cloud.dataflow.sql.ExecuteQueryServiceServer$SqlComplianceServiceImpl executeQuery
> SEVERE: !!!! Type mismatch:
> rowtype of new rel:
> RecordType(BIGINT distinct_2, BIGINT $col1, BIGINT $agg2) NOT NULL
> rowtype of set:
> RecordType(BIGINT distinct_2, BIGINT NOT NULL $col1, BIGINT NOT NULL $agg2) NOT NULL
> java.lang.AssertionError: Type mismatch:
> rowtype of new rel:
> RecordType(BIGINT distinct_2, BIGINT $col1, BIGINT $agg2) NOT NULL
> rowtype of set:
> RecordType(BIGINT distinct_2, BIGINT NOT NULL $col1, BIGINT NOT NULL $agg2) NOT NULL
> {code}



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