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)