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

Jark Wu edited comment on FLINK-18027 at 4/30/21, 8:40 AM:
-----------------------------------------------------------

Add a reproduce case: 

{code}
Flink SQL> CREATE TABLE Orders (
    order_number BIGINT,
    price        INT,
    first_name   STRING,
    last_name    STRING,
    buyer_name AS ROW(first_name, last_name)
) WITH (
  'connector' = 'datagen'
);

Flink SQL> select ROW(order_number, ROW(first_name, last_name)) from Orders;
[ERROR] Could not execute SQL statement. Reason:
org.apache.flink.sql.parser.impl.ParseException: Incorrect syntax near the 
keyword 'ROW' at line 1, column 26.
Was expecting one of:
    <BRACKET_QUOTED_IDENTIFIER> ...
    <QUOTED_IDENTIFIER> ...
    <BACK_QUOTED_IDENTIFIER> ...
    <HYPHENATED_IDENTIFIER> ...
    <IDENTIFIER> ...
    <UNICODE_QUOTED_IDENTIFIER> ...

        at 
org.apache.flink.sql.parser.impl.FlinkSqlParserImpl.convertException(FlinkSqlParserImpl.java:410)
        at 
org.apache.flink.sql.parser.impl.FlinkSqlParserImpl.normalizeException(FlinkSqlParserImpl.java:213)
        at 
org.apache.calcite.sql.parser.SqlParser.handleException(SqlParser.java:140)
        at 
org.apache.calcite.sql.parser.SqlParser.parseQuery(SqlParser.java:155)
        at org.apache.calcite.sql.parser.SqlParser.parseStmt(SqlParser.java:180)
        at 
org.apache.flink.table.planner.parse.CalciteParser.parse(CalciteParser.java:54)
        at 
org.apache.flink.table.planner.delegation.ParserImpl.parse(ParserImpl.java:96)
        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)
        at org.apache.flink.table.client.SqlClient.main(SqlClient.java:161)
{code}


was (Author: jark):
Add a reproduce case: 

{code}
Flink SQL> select ROW(order_number, ROW(first_name, last_name)) from Orders;
[ERROR] Could not execute SQL statement. Reason:
org.apache.flink.sql.parser.impl.ParseException: Incorrect syntax near the 
keyword 'ROW' at line 1, column 26.
Was expecting one of:
    <BRACKET_QUOTED_IDENTIFIER> ...
    <QUOTED_IDENTIFIER> ...
    <BACK_QUOTED_IDENTIFIER> ...
    <HYPHENATED_IDENTIFIER> ...
    <IDENTIFIER> ...
    <UNICODE_QUOTED_IDENTIFIER> ...

        at 
org.apache.flink.sql.parser.impl.FlinkSqlParserImpl.convertException(FlinkSqlParserImpl.java:410)
        at 
org.apache.flink.sql.parser.impl.FlinkSqlParserImpl.normalizeException(FlinkSqlParserImpl.java:213)
        at 
org.apache.calcite.sql.parser.SqlParser.handleException(SqlParser.java:140)
        at 
org.apache.calcite.sql.parser.SqlParser.parseQuery(SqlParser.java:155)
        at org.apache.calcite.sql.parser.SqlParser.parseStmt(SqlParser.java:180)
        at 
org.apache.flink.table.planner.parse.CalciteParser.parse(CalciteParser.java:54)
        at 
org.apache.flink.table.planner.delegation.ParserImpl.parse(ParserImpl.java:96)
        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)
        at org.apache.flink.table.client.SqlClient.main(SqlClient.java:161)
{code}

> ROW value constructor cannot deal with complex expressions
> ----------------------------------------------------------
>
>                 Key: FLINK-18027
>                 URL: https://issues.apache.org/jira/browse/FLINK-18027
>             Project: Flink
>          Issue Type: Bug
>          Components: Table SQL / API
>            Reporter: Benchao Li
>            Priority: Major
>              Labels: pull-request-available
>
> {code:java}
> create table my_source (
> my_row row<a int, b int, c int>
> ) with (...);
> create table my_sink (
> my_row row<a int, b int>
> ) with (...);
> insert into my_sink
> select ROW(my_row.a, my_row.b) 
> from my_source;{code}
> will throw excepions:
> {code:java}
> Exception in thread "main" org.apache.flink.table.api.SqlParserException: SQL 
> parse failed. Encountered "." at line 1, column 18.Exception in thread "main" 
> org.apache.flink.table.api.SqlParserException: SQL parse failed. Encountered 
> "." at line 1, column 18.Was expecting one of:    ")" ...    "," ...     at 
> org.apache.flink.table.planner.calcite.CalciteParser.parse(CalciteParser.java:56)
>  at 
> org.apache.flink.table.planner.delegation.ParserImpl.parse(ParserImpl.java:64)
>  at 
> org.apache.flink.table.api.internal.TableEnvironmentImpl.sqlQuery(TableEnvironmentImpl.java:627)
>  at com.bytedance.demo.KafkaTableSource.main(KafkaTableSource.java:76)Caused 
> by: org.apache.calcite.sql.parser.SqlParseException: Encountered "." at line 
> 1, column 18.Was expecting one of:    ")" ...    "," ...     at 
> org.apache.flink.sql.parser.impl.FlinkSqlParserImpl.convertException(FlinkSqlParserImpl.java:416)
>  at 
> org.apache.flink.sql.parser.impl.FlinkSqlParserImpl.normalizeException(FlinkSqlParserImpl.java:201)
>  at 
> org.apache.calcite.sql.parser.SqlParser.handleException(SqlParser.java:148) 
> at org.apache.calcite.sql.parser.SqlParser.parseQuery(SqlParser.java:163) at 
> org.apache.calcite.sql.parser.SqlParser.parseStmt(SqlParser.java:188) at 
> org.apache.flink.table.planner.calcite.CalciteParser.parse(CalciteParser.java:54)
>  ... 3 moreCaused by: org.apache.flink.sql.parser.impl.ParseException: 
> Encountered "." at line 1, column 18.Was expecting one of:    ")" ...    "," 
> ...     at 
> org.apache.flink.sql.parser.impl.FlinkSqlParserImpl.generateParseException(FlinkSqlParserImpl.java:36161)
>  at 
> org.apache.flink.sql.parser.impl.FlinkSqlParserImpl.jj_consume_token(FlinkSqlParserImpl.java:35975)
>  at 
> org.apache.flink.sql.parser.impl.FlinkSqlParserImpl.ParenthesizedSimpleIdentifierList(FlinkSqlParserImpl.java:21432)
>  at 
> org.apache.flink.sql.parser.impl.FlinkSqlParserImpl.Expression3(FlinkSqlParserImpl.java:17164)
>  at 
> org.apache.flink.sql.parser.impl.FlinkSqlParserImpl.Expression2b(FlinkSqlParserImpl.java:16820)
>  at 
> org.apache.flink.sql.parser.impl.FlinkSqlParserImpl.Expression2(FlinkSqlParserImpl.java:16861)
>  at 
> org.apache.flink.sql.parser.impl.FlinkSqlParserImpl.Expression(FlinkSqlParserImpl.java:16792)
>  at 
> org.apache.flink.sql.parser.impl.FlinkSqlParserImpl.SelectExpression(FlinkSqlParserImpl.java:11091)
>  at 
> org.apache.flink.sql.parser.impl.FlinkSqlParserImpl.SelectItem(FlinkSqlParserImpl.java:10293)
>  at 
> org.apache.flink.sql.parser.impl.FlinkSqlParserImpl.SelectList(FlinkSqlParserImpl.java:10267)
>  at 
> org.apache.flink.sql.parser.impl.FlinkSqlParserImpl.SqlSelect(FlinkSqlParserImpl.java:6943)
>  at 
> org.apache.flink.sql.parser.impl.FlinkSqlParserImpl.LeafQuery(FlinkSqlParserImpl.java:658)
>  at 
> org.apache.flink.sql.parser.impl.FlinkSqlParserImpl.LeafQueryOrExpr(FlinkSqlParserImpl.java:16775)
>  at 
> org.apache.flink.sql.parser.impl.FlinkSqlParserImpl.QueryOrExpr(FlinkSqlParserImpl.java:16238)
>  at 
> org.apache.flink.sql.parser.impl.FlinkSqlParserImpl.OrderedQueryOrExpr(FlinkSqlParserImpl.java:532)
>  at 
> org.apache.flink.sql.parser.impl.FlinkSqlParserImpl.SqlStmt(FlinkSqlParserImpl.java:3761)
>  at 
> org.apache.flink.sql.parser.impl.FlinkSqlParserImpl.SqlStmtEof(FlinkSqlParserImpl.java:3800)
>  at 
> org.apache.flink.sql.parser.impl.FlinkSqlParserImpl.parseSqlStmtEof(FlinkSqlParserImpl.java:248)
>  at org.apache.calcite.sql.parser.SqlParser.parseQuery(SqlParser.java:161) 
> ... 5 more
> {code}
>  



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to