[
https://issues.apache.org/jira/browse/FLINK-24352?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Caizhi Weng updated FLINK-24352:
--------------------------------
Description:
Add the following SQL to {{org.apache.flink.table.api.TableEnvironmentITCase}}
to reproduce this bug.
{code:scala}
@Test
def myTest(): Unit = {
tEnv.executeSql(
"""
|CREATE TABLE T1 (
| a INT,
| b STRING,
| proctime AS PROCTIME()
|) WITH (
| 'connector' = 'values',
| 'bounded' = 'true'
|)
|""".stripMargin)
tEnv.executeSql(
"""
|CREATE TABLE T2 (
| a INT,
| b STRING
|) WITH (
| 'connector' = 'values',
| 'bounded' = 'true'
|)
|""".stripMargin)
tEnv.explainSql(
"""
|WITH MyView(a, b) AS (SELECT a, b FROM T2)
|SELECT * FROM T1 AS T
|LEFT JOIN MyView FOR SYSTEM_TIME AS OF T.proctime AS D
|ON T.a = D.a
|""".stripMargin)
}
{code}
The exception stack is
{code}
org.apache.flink.table.api.ValidationException: SQL validation failed. null
at
org.apache.flink.table.planner.calcite.FlinkPlannerImpl.org$apache$flink$table$planner$calcite$FlinkPlannerImpl$$validate(FlinkPlannerImpl.scala:165)
at
org.apache.flink.table.planner.calcite.FlinkPlannerImpl.validate(FlinkPlannerImpl.scala:107)
at
org.apache.flink.table.planner.operations.SqlToOperationConverter.convert(SqlToOperationConverter.java:217)
at
org.apache.flink.table.planner.delegation.ParserImpl.parse(ParserImpl.java:101)
at
org.apache.flink.table.api.internal.TableEnvironmentImpl.explainSql(TableEnvironmentImpl.java:686)
// IDEA and Junit stacks are omitted
Caused by: java.lang.NullPointerException
at
org.apache.calcite.sql.validate.SqlValidatorImpl.validateSnapshot(SqlValidatorImpl.java:4714)
at
org.apache.calcite.sql.validate.SqlValidatorImpl.validateQuery(SqlValidatorImpl.java:986)
at
org.apache.calcite.sql.validate.SqlValidatorImpl.validateFrom(SqlValidatorImpl.java:3085)
at
org.apache.calcite.sql.validate.SqlValidatorImpl.validateFrom(SqlValidatorImpl.java:3070)
at
org.apache.calcite.sql.validate.SqlValidatorImpl.validateJoin(SqlValidatorImpl.java:3133)
at
org.apache.flink.table.planner.calcite.FlinkCalciteSqlValidator.validateJoin(FlinkCalciteSqlValidator.java:117)
at
org.apache.calcite.sql.validate.SqlValidatorImpl.validateFrom(SqlValidatorImpl.java:3076)
at
org.apache.calcite.sql.validate.SqlValidatorImpl.validateSelect(SqlValidatorImpl.java:3335)
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.validate.WithNamespace.validateImpl(WithNamespace.java:57)
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.validateWith(SqlValidatorImpl.java:3744)
at org.apache.calcite.sql.SqlWith.validate(SqlWith.java:71)
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:160)
... 43 more
{code}
However if we use {{CREATE VIEW}} statement to create this view it will run
successfully.
was:
Add the following SQL to {{org.apache.flink.table.api.TableEnvironmentITCase}}
to reproduce this bug.
{code:scala}
@Test
def myTest(): Unit = {
tEnv.executeSql(
"""
|CREATE TABLE T1 (
| a INT,
| b STRING,
| proctime AS PROCTIME()
|) WITH (
| 'connector' = 'values',
| 'bounded' = 'true'
|)
|""".stripMargin)
tEnv.executeSql(
"""
|CREATE TABLE T2 (
| a INT,
| b STRING
|) WITH (
| 'connector' = 'values',
| 'bounded' = 'true'
|)
|""".stripMargin)
tEnv.explainSql(
"""
|WITH MyView(a, b) AS (SELECT a, b FROM T2)
|SELECT * FROM T1 AS T
|LEFT JOIN MyView FOR SYSTEM_TIME AS OF T.proctime AS D
|ON T.a = D.a
|""".stripMargin)
}
{code}
The exception stack is
{code}
org.apache.flink.table.api.ValidationException: SQL validation failed. null
at
org.apache.flink.table.planner.calcite.FlinkPlannerImpl.org$apache$flink$table$planner$calcite$FlinkPlannerImpl$$validate(FlinkPlannerImpl.scala:165)
at
org.apache.flink.table.planner.calcite.FlinkPlannerImpl.validate(FlinkPlannerImpl.scala:107)
at
org.apache.flink.table.planner.operations.SqlToOperationConverter.convert(SqlToOperationConverter.java:217)
at
org.apache.flink.table.planner.delegation.ParserImpl.parse(ParserImpl.java:101)
at
org.apache.flink.table.api.internal.TableEnvironmentImpl.explainSql(TableEnvironmentImpl.java:686)
// IDEA and Junit stacks are omitted
Caused by: java.lang.NullPointerException
at
org.apache.calcite.sql.validate.SqlValidatorImpl.validateSnapshot(SqlValidatorImpl.java:4714)
at
org.apache.calcite.sql.validate.SqlValidatorImpl.validateQuery(SqlValidatorImpl.java:986)
at
org.apache.calcite.sql.validate.SqlValidatorImpl.validateFrom(SqlValidatorImpl.java:3085)
at
org.apache.calcite.sql.validate.SqlValidatorImpl.validateFrom(SqlValidatorImpl.java:3070)
at
org.apache.calcite.sql.validate.SqlValidatorImpl.validateJoin(SqlValidatorImpl.java:3133)
at
org.apache.flink.table.planner.calcite.FlinkCalciteSqlValidator.validateJoin(FlinkCalciteSqlValidator.java:117)
at
org.apache.calcite.sql.validate.SqlValidatorImpl.validateFrom(SqlValidatorImpl.java:3076)
at
org.apache.calcite.sql.validate.SqlValidatorImpl.validateSelect(SqlValidatorImpl.java:3335)
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.validate.WithNamespace.validateImpl(WithNamespace.java:57)
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.validateWith(SqlValidatorImpl.java:3744)
at org.apache.calcite.sql.SqlWith.validate(SqlWith.java:71)
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:160)
... 43 more
{code}
> Lookup join + WITH clause throws NPE
> ------------------------------------
>
> Key: FLINK-24352
> URL: https://issues.apache.org/jira/browse/FLINK-24352
> Project: Flink
> Issue Type: Bug
> Components: Table SQL / Planner
> Affects Versions: 1.15.0
> Reporter: Caizhi Weng
> Priority: Major
>
> Add the following SQL to
> {{org.apache.flink.table.api.TableEnvironmentITCase}} to reproduce this bug.
> {code:scala}
> @Test
> def myTest(): Unit = {
> tEnv.executeSql(
> """
> |CREATE TABLE T1 (
> | a INT,
> | b STRING,
> | proctime AS PROCTIME()
> |) WITH (
> | 'connector' = 'values',
> | 'bounded' = 'true'
> |)
> |""".stripMargin)
> tEnv.executeSql(
> """
> |CREATE TABLE T2 (
> | a INT,
> | b STRING
> |) WITH (
> | 'connector' = 'values',
> | 'bounded' = 'true'
> |)
> |""".stripMargin)
> tEnv.explainSql(
> """
> |WITH MyView(a, b) AS (SELECT a, b FROM T2)
> |SELECT * FROM T1 AS T
> |LEFT JOIN MyView FOR SYSTEM_TIME AS OF T.proctime AS D
> |ON T.a = D.a
> |""".stripMargin)
> }
> {code}
> The exception stack is
> {code}
> org.apache.flink.table.api.ValidationException: SQL validation failed. null
> at
> org.apache.flink.table.planner.calcite.FlinkPlannerImpl.org$apache$flink$table$planner$calcite$FlinkPlannerImpl$$validate(FlinkPlannerImpl.scala:165)
> at
> org.apache.flink.table.planner.calcite.FlinkPlannerImpl.validate(FlinkPlannerImpl.scala:107)
> at
> org.apache.flink.table.planner.operations.SqlToOperationConverter.convert(SqlToOperationConverter.java:217)
> at
> org.apache.flink.table.planner.delegation.ParserImpl.parse(ParserImpl.java:101)
> at
> org.apache.flink.table.api.internal.TableEnvironmentImpl.explainSql(TableEnvironmentImpl.java:686)
> // IDEA and Junit stacks are omitted
> Caused by: java.lang.NullPointerException
> at
> org.apache.calcite.sql.validate.SqlValidatorImpl.validateSnapshot(SqlValidatorImpl.java:4714)
> at
> org.apache.calcite.sql.validate.SqlValidatorImpl.validateQuery(SqlValidatorImpl.java:986)
> at
> org.apache.calcite.sql.validate.SqlValidatorImpl.validateFrom(SqlValidatorImpl.java:3085)
> at
> org.apache.calcite.sql.validate.SqlValidatorImpl.validateFrom(SqlValidatorImpl.java:3070)
> at
> org.apache.calcite.sql.validate.SqlValidatorImpl.validateJoin(SqlValidatorImpl.java:3133)
> at
> org.apache.flink.table.planner.calcite.FlinkCalciteSqlValidator.validateJoin(FlinkCalciteSqlValidator.java:117)
> at
> org.apache.calcite.sql.validate.SqlValidatorImpl.validateFrom(SqlValidatorImpl.java:3076)
> at
> org.apache.calcite.sql.validate.SqlValidatorImpl.validateSelect(SqlValidatorImpl.java:3335)
> 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.validate.WithNamespace.validateImpl(WithNamespace.java:57)
> 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.validateWith(SqlValidatorImpl.java:3744)
> at org.apache.calcite.sql.SqlWith.validate(SqlWith.java:71)
> 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:160)
> ... 43 more
> {code}
> However if we use {{CREATE VIEW}} statement to create this view it will run
> successfully.
--
This message was sent by Atlassian Jira
(v8.3.4#803005)