[
https://issues.apache.org/jira/browse/FLINK-18027?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17711200#comment-17711200
]
Aitozi edited comment on FLINK-18027 at 4/12/23 5:43 AM:
-
The content is modified
from
[https://github.com/apache/flink/blob/177310ebe3d552ec71a1f1f97e0207cf30b6efed/docs/dev/table/functions/systemFunctions.md]
to
[https://github.com/apache/flink/blob/a5372e0c92c7a0f465beba5e5204b07769cd92e6/docs/data/sql_functions.yml]
I think the content below is removed accidently. CC [~sjwiesman]
{code:java}
-- explicit ROW constructor ROW(value1 [, value2]*) {% endhighlight %} {code}
I think we should add this back, since the explicit ROW constructor's
limitation has been solved
was (Author: aitozi):
The content is modified
from
[https://github.com/apache/flink/blob/177310ebe3d552ec71a1f1f97e0207cf30b6efed/docs/dev/table/functions/systemFunctions.md]
to
[https://github.com/apache/flink/blob/a5372e0c92c7a0f465beba5e5204b07769cd92e6/docs/data/sql_functions.yml]
I think the content below is removed accidently.
{code:java}
-- explicit ROW constructor ROW(value1 [, value2]*) {% endhighlight %} {code}
I think we should add this back, since the explicit ROW constructor's
limitation has been solved
> 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: Not a Priority
> Labels: auto-deprioritized-major, auto-deprioritized-minor,
> pull-request-available
>
> {code:java}
> create table my_source (
> my_row row
> ) with (...);
> create table my_sink (
> my_row row
> ) 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
>