[
https://issues.apache.org/jira/browse/FLINK-39293?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Nic Townsend updated FLINK-39293:
---------------------------------
Description:
This appears to be a regression of FLINK-20077 - although I note the returned
exception/stack trace are wildly different.
{code}
final StreamExecutionEnvironment env =
StreamExecutionEnvironment.getExecutionEnvironment();
final StreamTableEnvironment tEnv = StreamTableEnvironment.create(env);
tEnv.executeSql("" +
"CREATE TEMPORARY TABLE data (\n" +
" id INT,\n" +
" ts AS PROCTIME()\n" +
") WITH (\n" +
" 'connector' = 'datagen',\n" +
" 'rows-per-second' = '3',\n" +
" 'fields.id.kind' = 'sequence',\n" +
" 'fields.id.start' = '1000000',\n" +
" 'fields.id.end' = '2000000'\n" +
")");
tEnv.executeSql("" +
"CREATE TEMPORARY VIEW events AS \n" +
"SELECT 1 AS key, id, MOD(id, 10) AS measurement, ts \n" +
"FROM data");
tEnv.executeSql("" +
"CREATE TEMPORARY VIEW foo AS \n" +
"SELECT * \n" +
"FROM events MATCH_RECOGNIZE (\n" +
" PARTITION BY key \n" +
" ORDER BY ts ASC \n" +
" MEASURES \n" +
" this_step.id as startId,\n" +
" next_step.id as nextId,\n" +
" this_step.ts AS ts1,\n" +
" next_step.ts AS ts2,\n" +
" next_step.measurement - this_step.measurement AS diff \n" +
" AFTER MATCH SKIP TO NEXT ROW \n" +
" PATTERN (this_step next_step)\n" +
" DEFINE this_step AS TRUE,\n" +
" next_step AS TRUE\n" +
")");
tEnv.executeSql("SELECT * FROM foo");
{code}
Returns the exception:
{code}
Exception in thread "main" org.apache.flink.table.api.SqlParserException: SQL
parse failed. Encountered "MATCH_RECOGNIZE" at line 2, column 64.
Was expecting one of:
<EOF>
"CROSS" ...
"EXCEPT" ...
"FETCH" ...
"FULL" ...
"GROUP" ...
"HAVING" ...
"INNER" ...
"INTERSECT" ...
"JOIN" ...
"LEFT" ...
"LIMIT" ...
"NATURAL" ...
"OFFSET" ...
"ORDER" ...
"OUTER" ...
"QUALIFY" ...
"RIGHT" ...
"MINUS" ...
"TABLESAMPLE" ...
"UNION" ...
"WHERE" ...
"WINDOW" ...
"(" ...
"," ...
at
org.apache.flink.table.planner.parse.CalciteParser.parse(CalciteParser.java:61)
at
org.apache.flink.table.planner.calcite.FlinkPlannerImpl$ToRelContextImpl.expandView(FlinkPlannerImpl.scala:367)
at
org.apache.calcite.plan.ViewExpanders$1.expandView(ViewExpanders.java:52)
at
org.apache.flink.table.planner.catalog.SqlCatalogViewTable.convertToRel(SqlCatalogViewTable.java:58)
at
org.apache.flink.table.planner.plan.schema.ExpandingPreparingTable.expand(ExpandingPreparingTable.java:70)
at
org.apache.flink.table.planner.plan.schema.ExpandingPreparingTable.toRel(ExpandingPreparingTable.java:57)
at
org.apache.calcite.sql2rel.SqlToRelConverter.toRel(SqlToRelConverter.java:4142)
at
org.apache.calcite.sql2rel.SqlToRelConverter.convertIdentifier(SqlToRelConverter.java:2997)
at
org.apache.calcite.sql2rel.SqlToRelConverter.convertFrom(SqlToRelConverter.java:2529)
at
org.apache.calcite.sql2rel.SqlToRelConverter.convertFrom(SqlToRelConverter.java:2435)
at
org.apache.calcite.sql2rel.SqlToRelConverter.convertFrom(SqlToRelConverter.java:2380)
at
org.apache.calcite.sql2rel.SqlToRelConverter.convertSelectImpl(SqlToRelConverter.java:758)
at
org.apache.calcite.sql2rel.SqlToRelConverter.convertSelect(SqlToRelConverter.java:746)
at
org.apache.calcite.sql2rel.SqlToRelConverter.convertQueryRecursive(SqlToRelConverter.java:3967)
at
org.apache.calcite.sql2rel.SqlToRelConverter.convertQuery(SqlToRelConverter.java:650)
at
org.apache.flink.table.planner.calcite.FlinkPlannerImpl.org$apache$flink$table$planner$calcite$FlinkPlannerImpl$$rel(FlinkPlannerImpl.scala:235)
at
org.apache.flink.table.planner.calcite.FlinkPlannerImpl.rel(FlinkPlannerImpl.scala:210)
at
org.apache.flink.table.planner.operations.SqlNodeConvertContext.toRelRoot(SqlNodeConvertContext.java:82)
at
org.apache.flink.table.planner.operations.converters.SqlQueryConverter.convertSqlNode(SqlQueryConverter.java:48)
at
org.apache.flink.table.planner.operations.converters.SqlNodeConverters.convertSqlNode(SqlNodeConverters.java:133)
at
org.apache.flink.table.planner.operations.SqlNodeToOperationConversion.convertValidatedSqlNode(SqlNodeToOperationConversion.java:231)
at
org.apache.flink.table.planner.operations.SqlNodeToOperationConversion.convert(SqlNodeToOperationConversion.java:221)
at
org.apache.flink.table.planner.delegation.ParserImpl.parse(ParserImpl.java:106)
at
org.apache.flink.table.api.internal.TableEnvironmentImpl.executeSql(TableEnvironmentImpl.java:937)
at com.example.UnionTablesExample.main(UnionTablesExample.java:63)
Caused by: org.apache.calcite.sql.parser.SqlParseException: Encountered
"MATCH_RECOGNIZE" at line 2, column 64.
Was expecting one of:
<EOF>
"CROSS" ...
"EXCEPT" ...
"FETCH" ...
"FULL" ...
"GROUP" ...
"HAVING" ...
"INNER" ...
"INTERSECT" ...
"JOIN" ...
"LEFT" ...
"LIMIT" ...
"NATURAL" ...
"OFFSET" ...
"ORDER" ...
"OUTER" ...
"QUALIFY" ...
"RIGHT" ...
"MINUS" ...
"TABLESAMPLE" ...
"UNION" ...
"WHERE" ...
"WINDOW" ...
"(" ...
"," ...
at
org.apache.flink.sql.parser.impl.FlinkSqlParserImpl.convertException(FlinkSqlParserImpl.java:558)
at
org.apache.flink.sql.parser.impl.FlinkSqlParserImpl.normalizeException(FlinkSqlParserImpl.java:304)
at
org.apache.calcite.sql.parser.SqlParser.handleException(SqlParser.java:159)
at
org.apache.calcite.sql.parser.SqlParser.parseQuery(SqlParser.java:174)
at org.apache.calcite.sql.parser.SqlParser.parseStmt(SqlParser.java:199)
at
org.apache.flink.table.planner.parse.CalciteParser.parse(CalciteParser.java:56)
... 24 more
Caused by: org.apache.flink.sql.parser.impl.ParseException: Encountered
"MATCH_RECOGNIZE" at line 2, column 64.
Was expecting one of:
<EOF>
"CROSS" ...
"EXCEPT" ...
"FETCH" ...
"FULL" ...
"GROUP" ...
"HAVING" ...
"INNER" ...
"INTERSECT" ...
"JOIN" ...
"LEFT" ...
"LIMIT" ...
"NATURAL" ...
"OFFSET" ...
"ORDER" ...
"OUTER" ...
"QUALIFY" ...
"RIGHT" ...
"MINUS" ...
"TABLESAMPLE" ...
"UNION" ...
"WHERE" ...
"WINDOW" ...
"(" ...
"," ...
at
org.apache.flink.sql.parser.impl.FlinkSqlParserImpl.generateParseException(FlinkSqlParserImpl.java:54079)
at
org.apache.flink.sql.parser.impl.FlinkSqlParserImpl.jj_consume_token(FlinkSqlParserImpl.java:53884)
at
org.apache.flink.sql.parser.impl.FlinkSqlParserImpl.SqlStmtEof(FlinkSqlParserImpl.java:4483)
at
org.apache.flink.sql.parser.impl.FlinkSqlParserImpl.parseSqlStmtEof(FlinkSqlParserImpl.java:352)
at
org.apache.calcite.sql.parser.SqlParser.parseQuery(SqlParser.java:172)
... 26 more
{code}
was:
This appears to be a regression of
https://issues.apache.org/jira/browse/FLINK-20077 - although I note the
returned exception/stack trace are wildly different.
{code}
final StreamExecutionEnvironment env =
StreamExecutionEnvironment.getExecutionEnvironment();
final StreamTableEnvironment tEnv = StreamTableEnvironment.create(env);
tEnv.executeSql("" +
"CREATE TEMPORARY TABLE data (\n" +
" id INT,\n" +
" ts AS PROCTIME()\n" +
") WITH (\n" +
" 'connector' = 'datagen',\n" +
" 'rows-per-second' = '3',\n" +
" 'fields.id.kind' = 'sequence',\n" +
" 'fields.id.start' = '1000000',\n" +
" 'fields.id.end' = '2000000'\n" +
")");
tEnv.executeSql("" +
"CREATE TEMPORARY VIEW events AS \n" +
"SELECT 1 AS key, id, MOD(id, 10) AS measurement, ts \n" +
"FROM data");
tEnv.executeSql("" +
"CREATE TEMPORARY VIEW foo AS \n" +
"SELECT * \n" +
"FROM events MATCH_RECOGNIZE (\n" +
" PARTITION BY key \n" +
" ORDER BY ts ASC \n" +
" MEASURES \n" +
" this_step.id as startId,\n" +
" next_step.id as nextId,\n" +
" this_step.ts AS ts1,\n" +
" next_step.ts AS ts2,\n" +
" next_step.measurement - this_step.measurement AS diff \n" +
" AFTER MATCH SKIP TO NEXT ROW \n" +
" PATTERN (this_step next_step)\n" +
" DEFINE this_step AS TRUE,\n" +
" next_step AS TRUE\n" +
")");
tEnv.executeSql("SELECT * FROM foo");
{code}
Returns the exception:
{code}
Exception in thread "main" org.apache.flink.table.api.SqlParserException: SQL
parse failed. Encountered "MATCH_RECOGNIZE" at line 2, column 64.
Was expecting one of:
<EOF>
"CROSS" ...
"EXCEPT" ...
"FETCH" ...
"FULL" ...
"GROUP" ...
"HAVING" ...
"INNER" ...
"INTERSECT" ...
"JOIN" ...
"LEFT" ...
"LIMIT" ...
"NATURAL" ...
"OFFSET" ...
"ORDER" ...
"OUTER" ...
"QUALIFY" ...
"RIGHT" ...
"MINUS" ...
"TABLESAMPLE" ...
"UNION" ...
"WHERE" ...
"WINDOW" ...
"(" ...
"," ...
at
org.apache.flink.table.planner.parse.CalciteParser.parse(CalciteParser.java:61)
at
org.apache.flink.table.planner.calcite.FlinkPlannerImpl$ToRelContextImpl.expandView(FlinkPlannerImpl.scala:367)
at
org.apache.calcite.plan.ViewExpanders$1.expandView(ViewExpanders.java:52)
at
org.apache.flink.table.planner.catalog.SqlCatalogViewTable.convertToRel(SqlCatalogViewTable.java:58)
at
org.apache.flink.table.planner.plan.schema.ExpandingPreparingTable.expand(ExpandingPreparingTable.java:70)
at
org.apache.flink.table.planner.plan.schema.ExpandingPreparingTable.toRel(ExpandingPreparingTable.java:57)
at
org.apache.calcite.sql2rel.SqlToRelConverter.toRel(SqlToRelConverter.java:4142)
at
org.apache.calcite.sql2rel.SqlToRelConverter.convertIdentifier(SqlToRelConverter.java:2997)
at
org.apache.calcite.sql2rel.SqlToRelConverter.convertFrom(SqlToRelConverter.java:2529)
at
org.apache.calcite.sql2rel.SqlToRelConverter.convertFrom(SqlToRelConverter.java:2435)
at
org.apache.calcite.sql2rel.SqlToRelConverter.convertFrom(SqlToRelConverter.java:2380)
at
org.apache.calcite.sql2rel.SqlToRelConverter.convertSelectImpl(SqlToRelConverter.java:758)
at
org.apache.calcite.sql2rel.SqlToRelConverter.convertSelect(SqlToRelConverter.java:746)
at
org.apache.calcite.sql2rel.SqlToRelConverter.convertQueryRecursive(SqlToRelConverter.java:3967)
at
org.apache.calcite.sql2rel.SqlToRelConverter.convertQuery(SqlToRelConverter.java:650)
at
org.apache.flink.table.planner.calcite.FlinkPlannerImpl.org$apache$flink$table$planner$calcite$FlinkPlannerImpl$$rel(FlinkPlannerImpl.scala:235)
at
org.apache.flink.table.planner.calcite.FlinkPlannerImpl.rel(FlinkPlannerImpl.scala:210)
at
org.apache.flink.table.planner.operations.SqlNodeConvertContext.toRelRoot(SqlNodeConvertContext.java:82)
at
org.apache.flink.table.planner.operations.converters.SqlQueryConverter.convertSqlNode(SqlQueryConverter.java:48)
at
org.apache.flink.table.planner.operations.converters.SqlNodeConverters.convertSqlNode(SqlNodeConverters.java:133)
at
org.apache.flink.table.planner.operations.SqlNodeToOperationConversion.convertValidatedSqlNode(SqlNodeToOperationConversion.java:231)
at
org.apache.flink.table.planner.operations.SqlNodeToOperationConversion.convert(SqlNodeToOperationConversion.java:221)
at
org.apache.flink.table.planner.delegation.ParserImpl.parse(ParserImpl.java:106)
at
org.apache.flink.table.api.internal.TableEnvironmentImpl.executeSql(TableEnvironmentImpl.java:937)
at com.example.UnionTablesExample.main(UnionTablesExample.java:63)
Caused by: org.apache.calcite.sql.parser.SqlParseException: Encountered
"MATCH_RECOGNIZE" at line 2, column 64.
Was expecting one of:
<EOF>
"CROSS" ...
"EXCEPT" ...
"FETCH" ...
"FULL" ...
"GROUP" ...
"HAVING" ...
"INNER" ...
"INTERSECT" ...
"JOIN" ...
"LEFT" ...
"LIMIT" ...
"NATURAL" ...
"OFFSET" ...
"ORDER" ...
"OUTER" ...
"QUALIFY" ...
"RIGHT" ...
"MINUS" ...
"TABLESAMPLE" ...
"UNION" ...
"WHERE" ...
"WINDOW" ...
"(" ...
"," ...
at
org.apache.flink.sql.parser.impl.FlinkSqlParserImpl.convertException(FlinkSqlParserImpl.java:558)
at
org.apache.flink.sql.parser.impl.FlinkSqlParserImpl.normalizeException(FlinkSqlParserImpl.java:304)
at
org.apache.calcite.sql.parser.SqlParser.handleException(SqlParser.java:159)
at
org.apache.calcite.sql.parser.SqlParser.parseQuery(SqlParser.java:174)
at org.apache.calcite.sql.parser.SqlParser.parseStmt(SqlParser.java:199)
at
org.apache.flink.table.planner.parse.CalciteParser.parse(CalciteParser.java:56)
... 24 more
Caused by: org.apache.flink.sql.parser.impl.ParseException: Encountered
"MATCH_RECOGNIZE" at line 2, column 64.
Was expecting one of:
<EOF>
"CROSS" ...
"EXCEPT" ...
"FETCH" ...
"FULL" ...
"GROUP" ...
"HAVING" ...
"INNER" ...
"INTERSECT" ...
"JOIN" ...
"LEFT" ...
"LIMIT" ...
"NATURAL" ...
"OFFSET" ...
"ORDER" ...
"OUTER" ...
"QUALIFY" ...
"RIGHT" ...
"MINUS" ...
"TABLESAMPLE" ...
"UNION" ...
"WHERE" ...
"WINDOW" ...
"(" ...
"," ...
at
org.apache.flink.sql.parser.impl.FlinkSqlParserImpl.generateParseException(FlinkSqlParserImpl.java:54079)
at
org.apache.flink.sql.parser.impl.FlinkSqlParserImpl.jj_consume_token(FlinkSqlParserImpl.java:53884)
at
org.apache.flink.sql.parser.impl.FlinkSqlParserImpl.SqlStmtEof(FlinkSqlParserImpl.java:4483)
at
org.apache.flink.sql.parser.impl.FlinkSqlParserImpl.parseSqlStmtEof(FlinkSqlParserImpl.java:352)
at
org.apache.calcite.sql.parser.SqlParser.parseQuery(SqlParser.java:172)
... 26 more
{code}
> SqlParserException if I select from a view that uses MATCH_RECOGNIZE
> --------------------------------------------------------------------
>
> Key: FLINK-39293
> URL: https://issues.apache.org/jira/browse/FLINK-39293
> Project: Flink
> Issue Type: Bug
> Components: Table SQL / Runtime
> Affects Versions: 2.2.0
> Reporter: Nic Townsend
> Priority: Major
>
> This appears to be a regression of FLINK-20077 - although I note the returned
> exception/stack trace are wildly different.
> {code}
> final StreamExecutionEnvironment env =
> StreamExecutionEnvironment.getExecutionEnvironment();
> final StreamTableEnvironment tEnv = StreamTableEnvironment.create(env);
> tEnv.executeSql("" +
> "CREATE TEMPORARY TABLE data (\n" +
> " id INT,\n" +
> " ts AS PROCTIME()\n" +
> ") WITH (\n" +
> " 'connector' = 'datagen',\n" +
> " 'rows-per-second' = '3',\n" +
> " 'fields.id.kind' = 'sequence',\n" +
> " 'fields.id.start' = '1000000',\n" +
> " 'fields.id.end' = '2000000'\n" +
> ")");
> tEnv.executeSql("" +
> "CREATE TEMPORARY VIEW events AS \n" +
> "SELECT 1 AS key, id, MOD(id, 10) AS measurement, ts \n" +
> "FROM data");
> tEnv.executeSql("" +
> "CREATE TEMPORARY VIEW foo AS \n" +
> "SELECT * \n" +
> "FROM events MATCH_RECOGNIZE (\n" +
> " PARTITION BY key \n" +
> " ORDER BY ts ASC \n" +
> " MEASURES \n" +
> " this_step.id as startId,\n" +
> " next_step.id as nextId,\n" +
> " this_step.ts AS ts1,\n" +
> " next_step.ts AS ts2,\n" +
> " next_step.measurement - this_step.measurement AS diff \n" +
> " AFTER MATCH SKIP TO NEXT ROW \n" +
> " PATTERN (this_step next_step)\n" +
> " DEFINE this_step AS TRUE,\n" +
> " next_step AS TRUE\n" +
> ")");
> tEnv.executeSql("SELECT * FROM foo");
> {code}
> Returns the exception:
> {code}
> Exception in thread "main" org.apache.flink.table.api.SqlParserException: SQL
> parse failed. Encountered "MATCH_RECOGNIZE" at line 2, column 64.
> Was expecting one of:
> <EOF>
> "CROSS" ...
> "EXCEPT" ...
> "FETCH" ...
> "FULL" ...
> "GROUP" ...
> "HAVING" ...
> "INNER" ...
> "INTERSECT" ...
> "JOIN" ...
> "LEFT" ...
> "LIMIT" ...
> "NATURAL" ...
> "OFFSET" ...
> "ORDER" ...
> "OUTER" ...
> "QUALIFY" ...
> "RIGHT" ...
> "MINUS" ...
> "TABLESAMPLE" ...
> "UNION" ...
> "WHERE" ...
> "WINDOW" ...
> "(" ...
> "," ...
>
> at
> org.apache.flink.table.planner.parse.CalciteParser.parse(CalciteParser.java:61)
> at
> org.apache.flink.table.planner.calcite.FlinkPlannerImpl$ToRelContextImpl.expandView(FlinkPlannerImpl.scala:367)
> at
> org.apache.calcite.plan.ViewExpanders$1.expandView(ViewExpanders.java:52)
> at
> org.apache.flink.table.planner.catalog.SqlCatalogViewTable.convertToRel(SqlCatalogViewTable.java:58)
> at
> org.apache.flink.table.planner.plan.schema.ExpandingPreparingTable.expand(ExpandingPreparingTable.java:70)
> at
> org.apache.flink.table.planner.plan.schema.ExpandingPreparingTable.toRel(ExpandingPreparingTable.java:57)
> at
> org.apache.calcite.sql2rel.SqlToRelConverter.toRel(SqlToRelConverter.java:4142)
> at
> org.apache.calcite.sql2rel.SqlToRelConverter.convertIdentifier(SqlToRelConverter.java:2997)
> at
> org.apache.calcite.sql2rel.SqlToRelConverter.convertFrom(SqlToRelConverter.java:2529)
> at
> org.apache.calcite.sql2rel.SqlToRelConverter.convertFrom(SqlToRelConverter.java:2435)
> at
> org.apache.calcite.sql2rel.SqlToRelConverter.convertFrom(SqlToRelConverter.java:2380)
> at
> org.apache.calcite.sql2rel.SqlToRelConverter.convertSelectImpl(SqlToRelConverter.java:758)
> at
> org.apache.calcite.sql2rel.SqlToRelConverter.convertSelect(SqlToRelConverter.java:746)
> at
> org.apache.calcite.sql2rel.SqlToRelConverter.convertQueryRecursive(SqlToRelConverter.java:3967)
> at
> org.apache.calcite.sql2rel.SqlToRelConverter.convertQuery(SqlToRelConverter.java:650)
> at
> org.apache.flink.table.planner.calcite.FlinkPlannerImpl.org$apache$flink$table$planner$calcite$FlinkPlannerImpl$$rel(FlinkPlannerImpl.scala:235)
> at
> org.apache.flink.table.planner.calcite.FlinkPlannerImpl.rel(FlinkPlannerImpl.scala:210)
> at
> org.apache.flink.table.planner.operations.SqlNodeConvertContext.toRelRoot(SqlNodeConvertContext.java:82)
> at
> org.apache.flink.table.planner.operations.converters.SqlQueryConverter.convertSqlNode(SqlQueryConverter.java:48)
> at
> org.apache.flink.table.planner.operations.converters.SqlNodeConverters.convertSqlNode(SqlNodeConverters.java:133)
> at
> org.apache.flink.table.planner.operations.SqlNodeToOperationConversion.convertValidatedSqlNode(SqlNodeToOperationConversion.java:231)
> at
> org.apache.flink.table.planner.operations.SqlNodeToOperationConversion.convert(SqlNodeToOperationConversion.java:221)
> at
> org.apache.flink.table.planner.delegation.ParserImpl.parse(ParserImpl.java:106)
> at
> org.apache.flink.table.api.internal.TableEnvironmentImpl.executeSql(TableEnvironmentImpl.java:937)
> at com.example.UnionTablesExample.main(UnionTablesExample.java:63)
> Caused by: org.apache.calcite.sql.parser.SqlParseException: Encountered
> "MATCH_RECOGNIZE" at line 2, column 64.
> Was expecting one of:
> <EOF>
> "CROSS" ...
> "EXCEPT" ...
> "FETCH" ...
> "FULL" ...
> "GROUP" ...
> "HAVING" ...
> "INNER" ...
> "INTERSECT" ...
> "JOIN" ...
> "LEFT" ...
> "LIMIT" ...
> "NATURAL" ...
> "OFFSET" ...
> "ORDER" ...
> "OUTER" ...
> "QUALIFY" ...
> "RIGHT" ...
> "MINUS" ...
> "TABLESAMPLE" ...
> "UNION" ...
> "WHERE" ...
> "WINDOW" ...
> "(" ...
> "," ...
>
> at
> org.apache.flink.sql.parser.impl.FlinkSqlParserImpl.convertException(FlinkSqlParserImpl.java:558)
> at
> org.apache.flink.sql.parser.impl.FlinkSqlParserImpl.normalizeException(FlinkSqlParserImpl.java:304)
> at
> org.apache.calcite.sql.parser.SqlParser.handleException(SqlParser.java:159)
> at
> org.apache.calcite.sql.parser.SqlParser.parseQuery(SqlParser.java:174)
> at
> org.apache.calcite.sql.parser.SqlParser.parseStmt(SqlParser.java:199)
> at
> org.apache.flink.table.planner.parse.CalciteParser.parse(CalciteParser.java:56)
> ... 24 more
> Caused by: org.apache.flink.sql.parser.impl.ParseException: Encountered
> "MATCH_RECOGNIZE" at line 2, column 64.
> Was expecting one of:
> <EOF>
> "CROSS" ...
> "EXCEPT" ...
> "FETCH" ...
> "FULL" ...
> "GROUP" ...
> "HAVING" ...
> "INNER" ...
> "INTERSECT" ...
> "JOIN" ...
> "LEFT" ...
> "LIMIT" ...
> "NATURAL" ...
> "OFFSET" ...
> "ORDER" ...
> "OUTER" ...
> "QUALIFY" ...
> "RIGHT" ...
> "MINUS" ...
> "TABLESAMPLE" ...
> "UNION" ...
> "WHERE" ...
> "WINDOW" ...
> "(" ...
> "," ...
>
> at
> org.apache.flink.sql.parser.impl.FlinkSqlParserImpl.generateParseException(FlinkSqlParserImpl.java:54079)
> at
> org.apache.flink.sql.parser.impl.FlinkSqlParserImpl.jj_consume_token(FlinkSqlParserImpl.java:53884)
> at
> org.apache.flink.sql.parser.impl.FlinkSqlParserImpl.SqlStmtEof(FlinkSqlParserImpl.java:4483)
> at
> org.apache.flink.sql.parser.impl.FlinkSqlParserImpl.parseSqlStmtEof(FlinkSqlParserImpl.java:352)
> at
> org.apache.calcite.sql.parser.SqlParser.parseQuery(SqlParser.java:172)
> ... 26 more
> {code}
--
This message was sent by Atlassian Jira
(v8.20.10#820010)