[ https://issues.apache.org/jira/browse/FLINK-20539?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17854141#comment-17854141 ]
Pouria Pirzadeh edited comment on FLINK-20539 at 6/11/24 6:08 PM: ------------------------------------------------------------------ [~martijnvisser] Thanks for looking into the issue and merging the fix. I tried below query on 1.19 and I am still hitting (a similar) issue (See below). I need to use CAST so I can create a Row with named fields. The query works fine without CAST, so I believe the same issue (i.e. Data type mismatch between Calcite and Flink's type factory) is happening here too. {code:java} Table t1 = tableEnv.sqlQuery( "SELECT " + "CAST(ROW(name, price) AS ROW<name_val STRING, price_val BIGINT>) AS col " + "FROM orders"); tableEnv.createTemporaryView("t1", t1); tableEnv.sqlQuery("SELECT * FROM t1").execute().print(); {code} Here is the error: {code:java} Exception in thread "main" java.lang.AssertionError: Conversion to relational algebra failed to preserve datatypes: validated type: RecordType(RecordType:peek_no_expand(VARCHAR(2147483647) CHARACTER SET "UTF-16LE" name_val, BIGINT price_val) NOT NULL col) NOT NULL converted type: RecordType(RecordType(VARCHAR(2147483647) CHARACTER SET "UTF-16LE" name_val, BIGINT price_val) NOT NULL col) NOT NULL rel: LogicalProject(col=[$0]) LogicalProject(col=[CAST(ROW($1, $2)):RecordType(VARCHAR(2147483647) CHARACTER SET "UTF-16LE" name_val, BIGINT price_val) NOT NULL]) LogicalProject(ts=[$2], name=[$0], price=[$1], rowtime=[$3]) LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[$3]) LogicalProject(name=[$0], price=[$1], ts=[$2], rowtime=[CAST($3):TIMESTAMP_LTZ(3) *ROWTIME*]) LogicalTableScan(table=[[anonymous_datastream_source$1, metadata=[rowtime]]]) {code} was (Author: pouria): [~martijnvisser] Thanks for looking into the issue and merging the fix. I tried below query on 1.19 and I am still hitting (a similar) issue (See below). I need to use CAST so I can create a Row with named fields. The query works fine without CAST, so I believe the same issue (i.e. Data type mismatch between Calcite and Flink's type factory) is happening here too. ``` Table t1 = tableEnv.sqlQuery( "SELECT " + "CAST(ROW(name, price) AS ROW<name_val STRING, price_val BIGINT>) AS col " + "FROM orders"); tableEnv.createTemporaryView("t1", t1); tableEnv.sqlQuery("SELECT * FROM t1").execute().print(); ``` Here is the error: ``` Exception in thread "main" java.lang.AssertionError: Conversion to relational algebra failed to preserve datatypes: validated type: RecordType(RecordType:peek_no_expand(VARCHAR(2147483647) CHARACTER SET "UTF-16LE" name_val, BIGINT price_val) NOT NULL col) NOT NULL converted type: RecordType(RecordType(VARCHAR(2147483647) CHARACTER SET "UTF-16LE" name_val, BIGINT price_val) NOT NULL col) NOT NULL rel: LogicalProject(col=[$0]) LogicalProject(col=[CAST(ROW($1, $2)):RecordType(VARCHAR(2147483647) CHARACTER SET "UTF-16LE" name_val, BIGINT price_val) NOT NULL]) LogicalProject(ts=[$2], name=[$0], price=[$1], rowtime=[$3]) LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[$3]) LogicalProject(name=[$0], price=[$1], ts=[$2], rowtime=[CAST($3):TIMESTAMP_LTZ(3) *ROWTIME*]) LogicalTableScan(table=[[*anonymous_datastream_source$1*, metadata=[rowtime]]]) ``` > 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: xuyang > Priority: Major > Labels: auto-unassigned, pull-request-available > Fix For: 1.19.0, 1.18.2 > > > 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)