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

Jark Wu commented on FLINK-20539:
---------------------------------

{code}
Exception in thread "main" org.apache.flink.table.client.SqlClientException: 
Unexpected exception. This is a bug. Please consider filing an issue.
        at 
org.apache.flink.table.client.SqlClient.startClient(SqlClient.java:201)
        at org.apache.flink.table.client.SqlClient.main(SqlClient.java:161)
Caused by: 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)
        at 
org.apache.flink.table.planner.calcite.FlinkPlannerImpl.org$apache$flink$table$planner$calcite$FlinkPlannerImpl$$rel(FlinkPlannerImpl.scala:168)
        at 
org.apache.flink.table.planner.calcite.FlinkPlannerImpl.rel(FlinkPlannerImpl.scala:160)
        at 
org.apache.flink.table.planner.operations.SqlToOperationConverter.toQueryOperation(SqlToOperationConverter.java:967)
        at 
org.apache.flink.table.planner.operations.SqlToOperationConverter.convertSqlQuery(SqlToOperationConverter.java:936)
        at 
org.apache.flink.table.planner.operations.SqlToOperationConverter.convert(SqlToOperationConverter.java:275)
        at 
org.apache.flink.table.planner.delegation.ParserImpl.parse(ParserImpl.java:99)
        at 
org.apache.flink.table.client.gateway.local.LocalExecutor.lambda$parseStatement$1(LocalExecutor.java:176)
        at 
org.apache.flink.table.client.gateway.context.ExecutionContext.wrapClassLoader(ExecutionContext.java:90)
        at 
org.apache.flink.table.client.gateway.local.LocalExecutor.parseStatement(LocalExecutor.java:176)
        at 
org.apache.flink.table.client.cli.CliClient.parseCommand(CliClient.java:385)
        at 
org.apache.flink.table.client.cli.CliClient.executeStatement(CliClient.java:326)
        at 
org.apache.flink.table.client.cli.CliClient.executeInteractive(CliClient.java:297)
        at 
org.apache.flink.table.client.cli.CliClient.executeInInteractiveMode(CliClient.java:221)
        at org.apache.flink.table.client.SqlClient.openCli(SqlClient.java:151)
        at org.apache.flink.table.client.SqlClient.start(SqlClient.java:95)
        at 
org.apache.flink.table.client.SqlClient.startClient(SqlClient.java:187)
        ... 1 more
{code}

> 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
>            Priority: Major
>              Labels: auto-unassigned
>
> 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.3.4#803005)

Reply via email to