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/04/01 22:21:00 UTC

[jira] [Created] (BEAM-9663) BeamSortRule NullPointerException offset parameter

Andrew Pilloud created BEAM-9663:
------------------------------------

             Summary: BeamSortRule NullPointerException offset parameter
                 Key: BEAM-9663
                 URL: https://issues.apache.org/jira/browse/BEAM-9663
             Project: Beam
          Issue Type: Bug
          Components: dsl-sql-zetasql
            Reporter: Andrew Pilloud


one failure in shard 37
{code}
Apr 01, 2020 11:51:04 AM cloud.dataflow.sql.ExecuteQueryServiceServer$SqlComplianceServiceImpl executeQuery
SEVERE: !!!! Error while applying rule BeamSortRule, args [rel#6203:LogicalSort.NONE(input=RelSubset#6202,offset=null:BIGINT,fetch=1:BIGINT)]
java.lang.RuntimeException: Error while applying rule BeamSortRule, args [rel#6203:LogicalSort.NONE(input=RelSubset#6202,offset=null:BIGINT,fetch=1:BIGINT)]
	at org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.volcano.VolcanoRuleCall.onMatch(VolcanoRuleCall.java:235)
	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)
Caused by: java.lang.NullPointerException
	at org.apache.beam.sdk.extensions.sql.impl.rel.BeamSortRel.<init>(BeamSortRel.java:136)
	at org.apache.beam.sdk.extensions.sql.impl.rule.BeamSortRule.convert(BeamSortRule.java:44)
	at org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.convert.ConverterRule.onMatch(ConverterRule.java:139)
	at org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.volcano.VolcanoRuleCall.onMatch(VolcanoRuleCall.java:208)
	... 15 more
{code}
{code}
Apr 01, 2020 11:51:03 AM cloud.dataflow.sql.ExecuteQueryServiceServer$SqlComplianceServiceImpl executeQuery
INFO: Processing Sql statement: SELECT a FROM (SELECT 1 a) LIMIT 1 OFFSET @ofst
{code}



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