You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by "Jane Chan (Jira)" <ji...@apache.org> on 2023/04/18 02:46:00 UTC

[jira] [Commented] (FLINK-31830) Coalesce on nested fields with different nullabilities will get wrong plan

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

Jane Chan commented on FLINK-31830:
-----------------------------------

I'd like to take this ticket. cc [~lincoln.86xy] 

> Coalesce on nested fields with different nullabilities will get wrong plan
> --------------------------------------------------------------------------
>
>                 Key: FLINK-31830
>                 URL: https://issues.apache.org/jira/browse/FLINK-31830
>             Project: Flink
>          Issue Type: Bug
>          Components: Table SQL / Planner
>    Affects Versions: 1.14.6
>            Reporter: lincoln lee
>            Priority: Major
>
> A test case similar to FLINK-31829, only changes the nullable field `a.np` to not null, will get a wrong plan in 1.14.x (reported from the community user):
> {code}
>   @Test
>   def testCoalesceOnNestedColumns(): Unit = {
>     val tEnv = util.tableEnv
>     val tableDescriptor = TableDescriptor.forConnector("datagen")
>         .schema(Schema.newBuilder
>             .column("id", DataTypes.INT.notNull)
>             .column("a", DataTypes.ROW(DataTypes.FIELD("np", DataTypes.INT.notNull())).nullable)
>             .build)
>         .build
>     tEnv.createTemporaryTable("t1", tableDescriptor)
>     tEnv.createTemporaryTable("t2", tableDescriptor)
>     val res = tEnv.executeSql("EXPLAIN SELECT a.id, COALESCE(a.a.np, b.a.np) c1, IFNULL(a.a.np, b.a.np) c2 FROM t1 a left JOIN t2 b ON a.id=b.id where a.a is null or a.a.np is null")
>     res.print()
> }  
> == Abstract Syntax Tree ==
> LogicalProject(id=[$0], c1=[CAST($1.np):INTEGER], c2=[IFNULL($1.np, $3.np)])
> +- LogicalFilter(condition=[OR(IS NULL($1), IS NULL(CAST($1.np):INTEGER))])
>    +- LogicalJoin(condition=[=($0, $2)], joinType=[left])
>       :- LogicalTableScan(table=[[default_catalog, default_database, t1]])
>       +- LogicalTableScan(table=[[default_catalog, default_database, t2]])
> {code}
> the top project in the ast is wrong:  `LogicalProject(id=[$0], c1=[CAST($1.np):INTEGER], c2=[IFNULL($1.np, $3.np)])`, the `c1=[CAST($1.np):INTEGER]` relate to `COALESCE(a.a.np, b.a.np) c1` is incorrect,
> but this works fine when using sql ddl to create tables
> {code}
>   @Test
>   def testCoalesceOnNestedColumns2(): Unit = {
>     val tEnv = util.tableEnv
>     tEnv.executeSql(
>       s"""
>          |create temporary table t1 (
>          |  id int not null,
>          |  a row<np int not null>
>          |) with (
>          | 'connector' = 'datagen'
>          |)
>          |""".stripMargin)
>     tEnv.executeSql(
>       s"""
>          |create temporary table t2 (
>          |  id int not null,
>          |  a row<np int not null>
>          |) with (
>          | 'connector' = 'datagen'
>          |)
>          |""".stripMargin)
>     val res = tEnv.executeSql(
>       "EXPLAIN SELECT a.id, COALESCE(a.a.np, b.a.np) c1, IFNULL(a.a.np, b.a.np) c2 FROM t1 a left JOIN t2 b ON a.id=b.id where a.a is null or a.a.np is null")
>     res.print()
>   }
> {code}
> from 1.15, the coalesce will be a new builtin function, and the ast looks correct in version 1.15+, while before 1.15 it was rewritten as `case when`



--
This message was sent by Atlassian Jira
(v8.20.10#820010)