[ https://issues.apache.org/jira/browse/BEAM-9698?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17110551#comment-17110551 ]
Andrew Pilloud commented on BEAM-9698: -------------------------------------- Per https://cloud.google.com/spanner/docs/data-types the NULL array and Empty array are both distinct and valid values. This matches the behavior of the reference implementation, BigQuery is explicitly turning a NULL array into an Empty array. The fix seems clear based on this (treat NULL as an empty array in uncollect). However there are all sorts of bugs here. The unparsed SQL is 'NULL', which produces an output of type INT64, value NULL. We don't actually look at the type and just assume it is an ARRAY<INT64>. This isn't a problem for NULL because we don't look at the type for anything and just return a java NULL. This has revealed a few areas for improvement in ZetaSqlCalc: 1. ZetaSQL Values contain a type, we don't use that type information when converting, instead we use the output schema. (zetaSqlValueToJavaObject shouldn't take Field Type.) 2. We should use PreparedExpression.getOutputType() in setup to validate the output type matches what we expect, there is a class of runtime bugs this would fix. cc: [~robinyqiu] > BeamUncollectRel UncollectDoFn NullPointerException > --------------------------------------------------- > > Key: BEAM-9698 > URL: https://issues.apache.org/jira/browse/BEAM-9698 > Project: Beam > Issue Type: Bug > Components: dsl-sql-zetasql > Reporter: Andrew Pilloud > Assignee: Andrew Pilloud > Priority: P2 > Labels: zetasql-compliance > > two failures in shard 19 > {code} > org.apache.beam.sdk.Pipeline$PipelineExecutionException: > java.lang.NullPointerException > at > org.apache.beam.runners.direct.DirectRunner$DirectPipelineResult.waitUntilFinish(DirectRunner.java:348) > at > org.apache.beam.runners.direct.DirectRunner$DirectPipelineResult.waitUntilFinish(DirectRunner.java:318) > at > org.apache.beam.runners.direct.DirectRunner.run(DirectRunner.java:213) > at org.apache.beam.runners.direct.DirectRunner.run(DirectRunner.java:67) > at org.apache.beam.sdk.Pipeline.run(Pipeline.java:317) > at org.apache.beam.sdk.Pipeline.run(Pipeline.java:303) > at > org.apache.beam.sdk.extensions.sql.impl.rel.BeamEnumerableConverter.runCollector(BeamEnumerableConverter.java:201) > at > org.apache.beam.sdk.extensions.sql.impl.rel.BeamEnumerableConverter.collectRows(BeamEnumerableConverter.java:218) > at > org.apache.beam.sdk.extensions.sql.impl.rel.BeamEnumerableConverter.toRowList(BeamEnumerableConverter.java:150) > at > org.apache.beam.sdk.extensions.sql.impl.rel.BeamEnumerableConverter.toRowList(BeamEnumerableConverter.java:127) > at > cloud.dataflow.sql.ExecuteQueryServiceServer$SqlComplianceServiceImpl.executeQuery(ExecuteQueryServiceServer.java:329) > 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.BeamUncollectRel$UncollectDoFn.process(BeamUncollectRel.java:103) > {code} > {code} > Apr 01, 2020 5:58:27 PM > cloud.dataflow.sql.ExecuteQueryServiceServer$SqlComplianceServiceImpl > executeQuery > INFO: Processing Sql statement: SELECT e FROM UNNEST(CAST(NULL AS > ARRAY<INT64>)) e > Apr 01, 2020 5:58:27 PM > cloud.dataflow.sql.ExecuteQueryServiceServer$SqlComplianceServiceImpl > executeQuery > INFO: Processing Sql statement: SELECT e FROM UNNEST(CAST(NULL AS > ARRAY<STRUCT<INT64>>)) e > {code} -- This message was sent by Atlassian Jira (v8.3.4#803005)