[ 
https://issues.apache.org/jira/browse/FLINK-20539?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Timo Walther updated FLINK-20539:
---------------------------------
    Release Note: Before Flink 2.2, row types defined in SQL e.g. `SELECT 
CAST(f AS ROW<i NOT NULL>)` did ignore the `NOT NULL` constraint. This 
was more aligned with the SQL standard but caused many type inconsistencies and 
cryptic error message when working on nested data. For example, it prevented 
using rows in computed columns or join keys. The new behavior takes the 
nullability into consideration. The config option 
`table.legacy-nested-row-nullability` allows to restore the old behavior if 
required, but it is recommended to update existing queries that ignored 
constraints before.   (was: Before Flink 2.2, row types defined in SQL e.g. 
`SELECT CAST(f AS ROW<i NOT NULL>)` did ignore the `NOT NULL` constraint. 
This was more aligned with the SQL standard but caused many type 
inconsistencies and cryptic error message when working on nested data. For 
example, it prevented using rows in computed columns or join keys. The new 
behavior takes the nullability into consideration.)

> Type mismatch when using ROW in computed column
> -----------------------------------------------
>
>                 Key: FLINK-20539
>                 URL: https://issues.apache.org/jira/browse/FLINK-20539
>             Project: Flink
>          Issue Type: Sub-task
>          Components: Table SQL / API
>            Reporter: Timo Walther
>            Assignee: Sergey Nuyanzin
>            Priority: Major
>              Labels: auto-unassigned, pull-request-available
>             Fix For: 1.19.0, 1.18.2, 2.2.0
>
>
> The following SQL:
> {code}
> env.executeSql(
>       "CREATE TABLE Orders (\n"
>       + "    order_number BIGINT,\n"
>       + "    price        INT,\n"
>       + "    first_name   STRING,\n"
>       + "    last_name    STRING,\n"
>       + "    buyer_name AS ROW(first_name, last_name)\n"
>       + ") WITH (\n"
>       + "  'connector' = 'datagen'\n"
>       + ")");
> env.executeSql("SELECT * FROM Orders").print();
> {code}
> Fails with:
> {code}
> Exception in thread "main" java.lang.AssertionError: Conversion to relational 
> algebra failed to preserve datatypes:
> validated type:
> RecordType(BIGINT order_number, INTEGER price, VARCHAR(2147483647) CHARACTER 
> SET "UTF-16LE" first_name, VARCHAR(2147483647) CHARACTER SET "UTF-16LE" 
> last_name, RecordType:peek_no_expand(VARCHAR(2147483647) CHARACTER SET 
> "UTF-16LE" EXPR$0, VARCHAR(2147483647) CHARACTER SET "UTF-16LE" EXPR$1) NOT 
> NULL buyer_name) NOT NULL
> converted type:
> RecordType(BIGINT order_number, INTEGER price, VARCHAR(2147483647) CHARACTER 
> SET "UTF-16LE" first_name, VARCHAR(2147483647) CHARACTER SET "UTF-16LE" 
> last_name, RecordType(VARCHAR(2147483647) CHARACTER SET "UTF-16LE" EXPR$0, 
> VARCHAR(2147483647) CHARACTER SET "UTF-16LE" EXPR$1) NOT NULL buyer_name) NOT 
> NULL
> rel:
> LogicalProject(order_number=[$0], price=[$1], first_name=[$2], 
> last_name=[$3], buyer_name=[ROW($2, $3)])
>   LogicalTableScan(table=[[default_catalog, default_database, Orders]])
>       at 
> org.apache.calcite.sql2rel.SqlToRelConverter.checkConvertedType(SqlToRelConverter.java:467)
>       at 
> org.apache.calcite.sql2rel.SqlToRelConverter.convertQuery(SqlToRelConverter.java:582)
> {code}



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

Reply via email to