[ 
https://issues.apache.org/jira/browse/FLINK-36783?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

ASF GitHub Bot updated FLINK-36783:
-----------------------------------
    Labels: pull-request-available  (was: )

> CTAS from VALUES throws ValidatioonException
> --------------------------------------------
>
>                 Key: FLINK-36783
>                 URL: https://issues.apache.org/jira/browse/FLINK-36783
>             Project: Flink
>          Issue Type: Bug
>          Components: Table SQL / API
>    Affects Versions: 2.0-preview
>            Reporter: Yunfeng Zhou
>            Priority: Major
>              Labels: pull-request-available
>
> {code:java}
> @Test
>   def test(): Unit = {
>     tEnv.executeSql(s"""
>                        |create table myt as select b, c, d from
>                        |  (values
>                        |    (1, 1, 2, 'd1'),
>                        |    (2, 1, 2, 'd2'),
>                        |    (3, 2, 3, 'd3')
>                        |  ) as V(a, b, c, d)
>                        |  order by a
>                        |""".stripMargin)
>     tEnv.executeSql(s"""
>                        |create table snk (
>                        |  b INT,
>                        |  c INT,
>                        |  d STRING
>                        |) with (
>                        |  'connector' = 'values'
>                        |)
>                        |""".stripMargin)
>     println(tEnv.explainSql("insert into snk select * from myt"))
>   }
> {code}
> The code above throws the following exception
> {code}
> org.apache.flink.table.api.ValidationException: SQL validation failed. At 
> line 8, column 12: Column 'a' not found in any table
>       at 
> org.apache.flink.table.planner.calcite.FlinkPlannerImpl.org$apache$flink$table$planner$calcite$FlinkPlannerImpl$$validate(FlinkPlannerImpl.scala:204)
>       at 
> org.apache.flink.table.planner.calcite.FlinkPlannerImpl.validate(FlinkPlannerImpl.scala:117)
>       at 
> org.apache.flink.table.planner.operations.SqlNodeToOperationConversion.convert(SqlNodeToOperationConversion.java:258)
>       at 
> org.apache.flink.table.planner.operations.MergeTableAsUtil.maybeRewriteQuery(MergeTableAsUtil.java:151)
>       at 
> org.apache.flink.table.planner.operations.SqlCreateTableConverter.convertCreateTableAS(SqlCreateTableConverter.java:124)
>       at 
> org.apache.flink.table.planner.operations.SqlNodeToOperationConversion.convertValidatedSqlNode(SqlNodeToOperationConversion.java:306)
>       at 
> org.apache.flink.table.planner.operations.SqlNodeToOperationConversion.convert(SqlNodeToOperationConversion.java:259)
>       at 
> org.apache.flink.table.planner.delegation.ParserImpl.parse(ParserImpl.java:106)
>       at 
> org.apache.flink.table.api.internal.TableEnvironmentImpl.executeSql(TableEnvironmentImpl.java:728)
>       at 
> org.apache.flink.table.planner.runtime.batch.sql.CalcITCase.test(CalcITCase.scala:67)
>       at java.base/java.lang.reflect.Method.invoke(Method.java:566)
>       at 
> java.base/java.util.concurrent.RecursiveAction.exec(RecursiveAction.java:189)
>       at 
> java.base/java.util.concurrent.ForkJoinTask.doExec(ForkJoinTask.java:290)
>       at 
> java.base/java.util.concurrent.ForkJoinPool$WorkQueue.topLevelExec(ForkJoinPool.java:1020)
>       at 
> java.base/java.util.concurrent.ForkJoinPool.scan(ForkJoinPool.java:1656)
>       at 
> java.base/java.util.concurrent.ForkJoinPool.runWorker(ForkJoinPool.java:1594)
>       at 
> java.base/java.util.concurrent.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:183)
> Caused by: org.apache.calcite.runtime.CalciteContextException: At line 8, 
> column 12: Column 'a' not found in any table
>       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:505)
>       at org.apache.calcite.sql.SqlUtil.newContextException(SqlUtil.java:932)
>       at org.apache.calcite.sql.SqlUtil.newContextException(SqlUtil.java:917)
>       at 
> org.apache.calcite.sql.validate.SqlValidatorImpl.newValidationError(SqlValidatorImpl.java:5279)
>       at 
> org.apache.calcite.sql.validate.DelegatingScope.fullyQualify(DelegatingScope.java:273)
>       at 
> org.apache.calcite.sql.validate.OrderByScope.fullyQualify(OrderByScope.java:95)
>       at 
> org.apache.calcite.sql.validate.SqlValidatorImpl$Expander.visit(SqlValidatorImpl.java:6471)
>       at 
> org.apache.calcite.sql.validate.SqlValidatorImpl$Expander.visit(SqlValidatorImpl.java:6451)
>       at org.apache.calcite.sql.SqlIdentifier.accept(SqlIdentifier.java:324)
>       at 
> org.apache.calcite.sql.validate.SqlValidatorImpl$Expander.go(SqlValidatorImpl.java:6460)
>       at 
> org.apache.calcite.sql.validate.SqlValidatorImpl.expand(SqlValidatorImpl.java:6036)
>       at 
> org.apache.calcite.sql.validate.SqlValidatorImpl.validateOrderList(SqlValidatorImpl.java:4182)
>       at 
> org.apache.calcite.sql.validate.SqlValidatorImpl.validateSelect(SqlValidatorImpl.java:3649)
>       at 
> org.apache.calcite.sql.validate.SelectNamespace.validateImpl(SelectNamespace.java:64)
>       at 
> org.apache.calcite.sql.validate.AbstractNamespace.validate(AbstractNamespace.java:89)
>       at 
> org.apache.calcite.sql.validate.SqlValidatorImpl.validateNamespace(SqlValidatorImpl.java:1062)
>       at 
> org.apache.calcite.sql.validate.SqlValidatorImpl.validateQuery(SqlValidatorImpl.java:1037)
>       at org.apache.calcite.sql.SqlSelect.validate(SqlSelect.java:248)
>       at 
> org.apache.calcite.sql.validate.SqlValidatorImpl.validateScopedExpression(SqlValidatorImpl.java:1012)
>       at 
> org.apache.calcite.sql.validate.SqlValidatorImpl.validate(SqlValidatorImpl.java:761)
>       at 
> org.apache.flink.table.planner.calcite.FlinkPlannerImpl.org$apache$flink$table$planner$calcite$FlinkPlannerImpl$$validate(FlinkPlannerImpl.scala:200)
>       ... 16 more
> Caused by: org.apache.calcite.sql.validate.SqlValidatorException: Column 'a' 
> not found in any table
>       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:505)
>       at org.apache.calcite.runtime.Resources$ExInst.ex(Resources.java:599)
>       ... 36 more
> {code}



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to