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

Ingo Bürk commented on CALCITE-4653:
------------------------------------

I've debugged this issue a little bit further. The problem is that the query 
part of the CREATE VIEW statement runs through OrderedQueryOrExpr in the parser 
when Flink is expanding the statement. In there, "user" is seen as a keyword 
and thus transformed into "USER". Finally this fails in 
DelegatingScope#fullyQualify because findQualifyingTableNames returns an empty 
map.

I haven't yet been able to reproduce this with a test in Calcite directly, 
though. The following test for SqlValidatorTest triggers the same code path 
with OrderedQueryOrExpr, but it doesn't go through DelegationScope and thus 
doesn't trigger the error (but presumably would otherwise).
{code:java}
@Test void test() {
  sql("INSERT INTO EMP SELECT 1, user, '', 1, CURRENT_TIME, 1, 1, 1, TRUE FROM 
(SELECT '' AS \"user\" FROM (VALUES (TRUE)))").ok();
}
{code}

> Improve exception message when identifiers using reserved keyword
> -----------------------------------------------------------------
>
>                 Key: CALCITE-4653
>                 URL: https://issues.apache.org/jira/browse/CALCITE-4653
>             Project: Calcite
>          Issue Type: Improvement
>            Reporter: Martijn Visser
>            Priority: Minor
>
> Originally posted at https://issues.apache.org/jira/browse/FLINK-22999 but 
> decided to create the ticket here because there's not much that Flink can do.
> When using a reserved keyword in a SQL query, such as `user` in the following 
> in Flink snippet without using backticks:
> {code:java}
>         final StreamTableEnvironment tableEnv = 
> StreamTableEnvironment.create(env);
>         final DataStream<Row> clickStream =
>                 env.fromElements(
>                         Row.of("Bob", "ORDER_ENTERED", "1623758400"),
>                         Row.of("Alice", "ORDER_ENTERED", "1623758700"),
>                         Row.of("Bob", "ADDRESS_ENTERED", "1623759000"));
>         final Table clickTable = 
> tableEnv.fromDataStream(clickStream).as("user","pageType", "ts");
>         tableEnv.createTemporaryView("ClickStream", clickTable);
>         tableEnv.executeSql("CREATE VIEW FirstPageVisits AS SELECT user, 
> pageType, ts FROM ClickStream WHERE pageType = 'ORDER_ENTERED'");
> clickTable.execute().print();
> {code}
> The Calcite parser throws the following error:
> {code:java}
> Exception in thread "main" org.apache.flink.table.api.ValidationException: 
> SQL validation failed. From line 1, column 8 to line 1, column 11: Column 
> 'USER' not found in any table; did you mean 'user'?
>       at 
> org.apache.flink.table.planner.calcite.FlinkPlannerImpl.org$apache$flink$table$planner$calcite$FlinkPlannerImpl$$validate(FlinkPlannerImpl.scala:164)
>       at 
> org.apache.flink.table.planner.calcite.FlinkPlannerImpl.validate(FlinkPlannerImpl.scala:107)
>       at 
> org.apache.flink.table.planner.utils.Expander.expanded(Expander.java:86)
>       at 
> org.apache.flink.table.planner.operations.SqlToOperationConverter.convertViewQuery(SqlToOperationConverter.java:898)
>       at 
> org.apache.flink.table.planner.operations.SqlToOperationConverter.convertCreateView(SqlToOperationConverter.java:868)
>       at 
> org.apache.flink.table.planner.operations.SqlToOperationConverter.convert(SqlToOperationConverter.java:261)
>       at 
> org.apache.flink.table.planner.delegation.ParserImpl.parse(ParserImpl.java:101)
>       at 
> org.apache.flink.table.api.internal.TableEnvironmentImpl.executeSql(TableEnvironmentImpl.java:730)
>       at 
> org.apache.flink.table.examples.java.basics.StreamSQLExample.main(StreamSQLExample.java:69)
> Caused by: org.apache.calcite.runtime.CalciteContextException: From line 1, 
> column 8 to line 1, column 11: Column 'USER' not found in any table; did you 
> mean 'user'?
>       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.sql.SqlUtil.newContextException(SqlUtil.java:883)
>       at org.apache.calcite.sql.SqlUtil.newContextException(SqlUtil.java:868)
>       at 
> org.apache.calcite.sql.validate.SqlValidatorImpl.newValidationError(SqlValidatorImpl.java:4861)
>       at 
> org.apache.calcite.sql.validate.DelegatingScope.fullyQualify(DelegatingScope.java:254)
>       at 
> org.apache.calcite.sql.validate.SqlValidatorImpl$Expander.visit(SqlValidatorImpl.java:5833)
>       at 
> org.apache.calcite.sql.validate.SqlValidatorImpl$SelectExpander.visit(SqlValidatorImpl.java:5982)
>       at 
> org.apache.calcite.sql.validate.SqlValidatorImpl$SelectExpander.visit(SqlValidatorImpl.java:5967)
>       at org.apache.calcite.sql.SqlIdentifier.accept(SqlIdentifier.java:320)
>       at 
> org.apache.calcite.sql.validate.SqlValidatorImpl.expandSelectExpr(SqlValidatorImpl.java:5416)
>       at 
> org.apache.calcite.sql.validate.SqlValidatorImpl.expandSelectItem(SqlValidatorImpl.java:398)
>       at 
> org.apache.calcite.sql.validate.SqlValidatorImpl.validateSelectList(SqlValidatorImpl.java:4061)
>       at 
> org.apache.calcite.sql.validate.SqlValidatorImpl.validateSelect(SqlValidatorImpl.java:3347)
>       at 
> org.apache.calcite.sql.validate.SelectNamespace.validateImpl(SelectNamespace.java:60)
>       at 
> org.apache.calcite.sql.validate.AbstractNamespace.validate(AbstractNamespace.java:84)
>       at 
> org.apache.calcite.sql.validate.SqlValidatorImpl.validateNamespace(SqlValidatorImpl.java:997)
>       at 
> org.apache.calcite.sql.validate.SqlValidatorImpl.validateQuery(SqlValidatorImpl.java:975)
>       at org.apache.calcite.sql.SqlSelect.validate(SqlSelect.java:232)
>       at 
> org.apache.calcite.sql.validate.SqlValidatorImpl.validateScopedExpression(SqlValidatorImpl.java:952)
>       at 
> org.apache.calcite.sql.validate.SqlValidatorImpl.validate(SqlValidatorImpl.java:704)
>       at 
> org.apache.flink.table.planner.calcite.FlinkPlannerImpl.org$apache$flink$table$planner$calcite$FlinkPlannerImpl$$validate(FlinkPlannerImpl.scala:159)
>       ... 8 more
> Caused by: org.apache.calcite.sql.validate.SqlValidatorException: Column 
> 'USER' not found in any table; did you mean 'user'?
>       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)
>       ... 28 more
> {code}
> It would be helpful if the thrown exception would be more descriptive to the 
> end-user. 



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

Reply via email to