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

jibiyr commented on FLINK-20884:
--------------------------------

[~jark] [~danny0405] , it is still exists in master branch. 

I debug and found when flink resolve the table columns which has computed 
column, it call 

FlinkPlannerImpl#rex to convert computed column to rexNode. However, it does'nt 
call the findSubQuery.

so the npe occur.

 

> NullPointerException in create statements with computed columns which the 
> subQuery is SqlNodeList 
> --------------------------------------------------------------------------------------------------
>
>                 Key: FLINK-20884
>                 URL: https://issues.apache.org/jira/browse/FLINK-20884
>             Project: Flink
>          Issue Type: Bug
>          Components: Table SQL / Planner
>    Affects Versions: 1.12.0
>            Reporter: allan.hou
>            Priority: Major
>             Fix For: 1.13.0, 1.12.3
>
>
> Create statement:
> {code:java}
> CREATE TABLE hive.flink_hive.my_kafka_student (
> id INT,
> name VARCHAR,
> score INT,
> test as case when score in (1,2,3) then 1 else 0 end
> ) WITH (
> 'connector' = 'kafka',
> ...
> );
> {code}
> Then drop the table or run the application.It always fails with
> {code:java}
> Caused by: java.lang.NullPointerException
>         at java.util.Objects.requireNonNull(Objects.java:203) ~[?:1.8.0_262]
>         at 
> org.apache.calcite.sql2rel.SqlToRelConverter$Blackboard.convertExpression(SqlToRelConverter.java:4914)
>  ~[flink-table_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT]
>         at 
> org.apache.calcite.sql2rel.StandardConvertletTable.convertExpressionList(StandardConvertletTable.java:839)
>  ~[flink-table_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT]
>         at 
> org.apache.calcite.sql2rel.StandardConvertletTable.convertCall(StandardConvertletTable.java:815)
>  ~[flink-table_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT]
>         at 
> org.apache.calcite.sql2rel.StandardConvertletTable.convertCall(StandardConvertletTable.java:802)
>  ~[flink-table_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT]
>         at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) 
> ~[?:1.8.0_262]
>         at 
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) 
> ~[?:1.8.0_262]
>         at 
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>  ~[?:1.8.0_262]
>         at java.lang.reflect.Method.invoke(Method.java:498) ~[?:1.8.0_262]
>         at 
> org.apache.calcite.sql2rel.ReflectiveConvertletTable.lambda$registerNodeTypeMethod$0(ReflectiveConvertletTable.java:83)
>  ~[flink-table_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT]
>         at 
> org.apache.calcite.sql2rel.SqlNodeToRexConverterImpl.convertCall(SqlNodeToRexConverterImpl.java:62)
>  ~[flink-table_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT]
>         at 
> org.apache.calcite.sql2rel.SqlToRelConverter$Blackboard.visit(SqlToRelConverter.java:5098)
>  ~[flink-table_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT]
>         at 
> org.apache.calcite.sql2rel.SqlToRelConverter$Blackboard.visit(SqlToRelConverter.java:4374)
>  ~[flink-table_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT]
>         at org.apache.calcite.sql.SqlCall.accept(SqlCall.java:139) 
> ~[flink-table_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT]
>         at 
> org.apache.calcite.sql2rel.SqlToRelConverter$Blackboard.convertExpression(SqlToRelConverter.java:4961)
>  ~[flink-table_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT]
>         at 
> org.apache.calcite.sql2rel.StandardConvertletTable.convertCase(StandardConvertletTable.java:387)
>  ~[flink-table_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT]
>         at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) 
> ~[?:1.8.0_262]
>         at 
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) 
> ~[?:1.8.0_262]
>         at 
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>  ~[?:1.8.0_262]
>         at java.lang.reflect.Method.invoke(Method.java:498) ~[?:1.8.0_262]
>         at 
> org.apache.calcite.sql2rel.ReflectiveConvertletTable.lambda$registerNodeTypeMethod$0(ReflectiveConvertletTable.java:83)
>  ~[flink-table_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT]
>         at 
> org.apache.calcite.sql2rel.SqlNodeToRexConverterImpl.convertCall(SqlNodeToRexConverterImpl.java:62)
>  ~[flink-table_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT]
>         at 
> org.apache.calcite.sql2rel.SqlToRelConverter$Blackboard.visit(SqlToRelConverter.java:5098)
>  ~[flink-table_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT]
>         at 
> org.apache.calcite.sql2rel.SqlToRelConverter$Blackboard.visit(SqlToRelConverter.java:4374)
>  ~[flink-table_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT]
>         at org.apache.calcite.sql.SqlCall.accept(SqlCall.java:139) 
> ~[flink-table_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT]
>         at 
> org.apache.calcite.sql2rel.SqlToRelConverter$Blackboard.convertExpression(SqlToRelConverter.java:4961)
>  ~[flink-table_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT]
>         at 
> org.apache.calcite.sql2rel.SqlToRelConverter.convertExpression(SqlToRelConverter.java:1925)
>  ~[flink-table_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT]
>         at 
> org.apache.flink.table.planner.calcite.FlinkPlannerImpl.rex(FlinkPlannerImpl.scala:206)
>  ~[flink-table-blink_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT]
>         at 
> org.apache.flink.table.planner.calcite.FlinkPlannerImpl.rex(FlinkPlannerImpl.scala:180)
>  ~[flink-table-blink_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT]
>         at 
> org.apache.flink.table.planner.calcite.SqlExprToRexConverterImpl.lambda$convertToRexNodes$1(SqlExprToRexConverterImpl.java:68)
>  ~[flink-table-blink_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT]
>         at 
> java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:193) 
> ~[?:1.8.0_262]
>         at 
> java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:193) 
> ~[?:1.8.0_262]
>         at 
> java.util.Spliterators$ArraySpliterator.forEachRemaining(Spliterators.java:948)
>  ~[?:1.8.0_262]
>         at 
> java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:482) 
> ~[?:1.8.0_262]
>         at 
> java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:472) 
> ~[?:1.8.0_262]
>         at 
> java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:546) 
> ~[?:1.8.0_262]
>         at 
> java.util.stream.AbstractPipeline.evaluateToArrayNode(AbstractPipeline.java:260)
>  ~[?:1.8.0_262]
>         at 
> java.util.stream.ReferencePipeline.toArray(ReferencePipeline.java:505) 
> ~[?:1.8.0_262]
>         at 
> org.apache.flink.table.planner.calcite.SqlExprToRexConverterImpl.convertToRexNodes(SqlExprToRexConverterImpl.java:69)
>  ~[flink-table-blink_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT]
>         at 
> org.apache.flink.table.planner.calcite.SqlExprToRexConverterImpl.convertToRexNode(SqlExprToRexConverterImpl.java:60)
>  ~[flink-table-blink_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT]
>         at 
> org.apache.flink.table.planner.delegation.ParserImpl.parseSqlExpression(ParserImpl.java:93)
>  ~[flink-table-blink_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT]
>         at 
> org.apache.flink.table.api.internal.CatalogTableSchemaResolver.resolveExpressionDataType(CatalogTableSchemaResolver.java:137)
>  ~[flink-table_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT]
>         at 
> org.apache.flink.table.api.internal.CatalogTableSchemaResolver.resolve(CatalogTableSchemaResolver.java:87)
>  ~[flink-table_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT]
>         at 
> org.apache.flink.table.catalog.CatalogManager.resolveTableSchema(CatalogManager.java:380)
>  ~[flink-table_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT]
>         at 
> org.apache.flink.table.catalog.CatalogManager.getPermanentTable(CatalogManager.java:408)
>  ~[flink-table_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT]
>         at 
> org.apache.flink.table.catalog.CatalogManager.dropTableInternal(CatalogManager.java:760)
>  ~[flink-table_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT]
>         at 
> org.apache.flink.table.catalog.CatalogManager.dropTable(CatalogManager.java:725)
>  ~[flink-table_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT]
>         at 
> org.apache.flink.table.api.internal.TableEnvironmentImpl.executeOperation(TableEnvironmentImpl.java:781)
>  ~[flink-table_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT]
>         at 
> org.apache.flink.table.api.internal.TableEnvironmentImpl.executeSql(TableEnvironmentImpl.java:665)
>  ~[flink-table_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT]
>         at 
> org.apache.flink.table.client.gateway.local.LocalExecutor.lambda$executeSql$1(LocalExecutor.java:309)
>  ~[flink-sql-client_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT]
>         at 
> org.apache.flink.table.client.gateway.local.ExecutionContext.wrapClassLoader(ExecutionContext.java:257)
>  ~[flink-sql-client_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT]
>         at 
> org.apache.flink.table.client.gateway.local.LocalExecutor.executeSql(LocalExecutor.java:309)
>  ~[flink-sql-client_2.11-1.12-SNAPSHOT.jar:1.12-SNAPSHOT]
> {code}
>  
> It's fine in 1.11.0.  The 1.12 planner calls anthor calcite api to convert 
> SqlNode to RexNode. 
>  



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

Reply via email to