Sergey Nuyanzin created FLINK-26295:
---------------------------------------

             Summary: 'HELP ;', 'QUIT ;' and other sql-client commands fail 
with CalciteException: Non-query expression encountered in illegal context
                 Key: FLINK-26295
                 URL: https://issues.apache.org/jira/browse/FLINK-26295
             Project: Flink
          Issue Type: Bug
          Components: Table SQL / Client
            Reporter: Sergey Nuyanzin
             Fix For: 1.15.0


It seems the reason is https://github.com/apache/flink/pull/18363/files
where added condition like 
{code:java}
super(Pattern.compile("HELP;?", DEFAULT_PATTERN_FLAGS))
{code}
that means {{HELP;}} will work however if there is any space between {{HELP}} 
and {{;}} then not
and it fails like
{noformat}
[ERROR] Could not execute SQL statement. Reason:
org.apache.calcite.runtime.CalciteException: Non-query expression encountered 
in illegal context
        at 
java.base/jdk.internal.reflect.NativeConstructorAccessorImpl.newInstance0(Native
 Method)
        at 
java.base/jdk.internal.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:62)
        at 
java.base/jdk.internal.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)
        at 
java.base/java.lang.reflect.Constructor.newInstance(Constructor.java:490)
        at 
org.apache.calcite.runtime.Resources$ExInstWithCause.ex(Resources.java:467)
        at org.apache.calcite.runtime.Resources$ExInst.ex(Resources.java:560)
        at org.apache.calcite.sql.SqlUtil.newContextException(SqlUtil.java:883)
        at org.apache.calcite.sql.SqlUtil.newContextException(SqlUtil.java:868)
        at 
org.apache.flink.sql.parser.impl.FlinkSqlParserImpl.checkNonQueryExpression(FlinkSqlParserImpl.java:395)
        at 
org.apache.flink.sql.parser.impl.FlinkSqlParserImpl.Expression3(FlinkSqlParserImpl.java:21147)
        at 
org.apache.flink.sql.parser.impl.FlinkSqlParserImpl.Expression2b(FlinkSqlParserImpl.java:20816)
        at 
org.apache.flink.sql.parser.impl.FlinkSqlParserImpl.Expression2(FlinkSqlParserImpl.java:20857)
        at 
org.apache.flink.sql.parser.impl.FlinkSqlParserImpl.Expression(FlinkSqlParserImpl.java:20788)
        at 
org.apache.flink.sql.parser.impl.FlinkSqlParserImpl.LeafQueryOrExpr(FlinkSqlParserImpl.java:20765)
        at 
org.apache.flink.sql.parser.impl.FlinkSqlParserImpl.QueryOrExpr(FlinkSqlParserImpl.java:20213)
        at 
org.apache.flink.sql.parser.impl.FlinkSqlParserImpl.OrderedQueryOrExpr(FlinkSqlParserImpl.java:588)
        at 
org.apache.flink.sql.parser.impl.FlinkSqlParserImpl.SqlStmt(FlinkSqlParserImpl.java:3986)
        at 
org.apache.flink.sql.parser.impl.FlinkSqlParserImpl.SqlStmtList(FlinkSqlParserImpl.java:2915)
        at 
org.apache.flink.sql.parser.impl.FlinkSqlParserImpl.parseSqlStmtList(FlinkSqlParserImpl.java:287)
        at 
org.apache.calcite.sql.parser.SqlParser.parseStmtList(SqlParser.java:193)
        at 
org.apache.flink.table.planner.parse.CalciteParser.parseSqlList(CalciteParser.java:77)
        at 
org.apache.flink.table.planner.delegation.ParserImpl.parse(ParserImpl.java:101)
        at 
org.apache.flink.table.client.gateway.local.LocalExecutor.lambda$parseStatement$1(LocalExecutor.java:172)
        at 
org.apache.flink.table.client.gateway.context.ExecutionContext.wrapClassLoader(ExecutionContext.java:88)
        at 
org.apache.flink.table.client.gateway.local.LocalExecutor.parseStatement(LocalExecutor.java:172)
        at 
org.apache.flink.table.client.cli.SqlCommandParserImpl.parseCommand(SqlCommandParserImpl.java:45)
        at 
org.apache.flink.table.client.cli.SqlMultiLineParser.parse(SqlMultiLineParser.java:71)
        at 
org.jline.reader.impl.LineReaderImpl.acceptLine(LineReaderImpl.java:2731)
        at 
org.jline.reader.impl.LineReaderImpl.readLine(LineReaderImpl.java:585)
        at 
org.apache.flink.table.client.cli.CliClient.getAndExecuteStatements(CliClient.java:296)
        at 
org.apache.flink.table.client.cli.CliClient.executeInteractive(CliClient.java:281)
        at 
org.apache.flink.table.client.cli.CliClient.executeInInteractiveMode(CliClient.java:229)
        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)

{noformat}



--
This message was sent by Atlassian Jira
(v8.20.1#820001)

Reply via email to