Details
-
Sub-task
-
Status: Closed
-
Major
-
Resolution: Fixed
-
None
Description
If we have an INSERT INTO with an explicit column list and a TUMBLE function, the explicit column expansion fails with NullPointerException.
Test to reproduce:
@Test public void testExplicitTableWithinTableFunctionWithInsertIntoNamedColumns() { tableEnv.getConfig() .set( TABLE_COLUMN_EXPANSION_STRATEGY, Collections.singletonList(EXCLUDE_DEFAULT_VIRTUAL_METADATA_COLUMNS)); tableEnv.executeSql( "CREATE TABLE sink (\n" + " a STRING,\n" + " c BIGINT\n" + ") WITH (\n" + " 'connector' = 'values'," + " 'sink-insert-only' = 'false'" + ")"); tableEnv.explainSql( "INSERT INTO sink(a, c) " + "SELECT t3_s, COUNT(t3_i) FROM " + " TABLE(TUMBLE(TABLE t3, DESCRIPTOR(t3_m_virtual), INTERVAL '1' MINUTE)) " + "GROUP BY t3_s;"); }
Exception:
org.apache.flink.table.api.ValidationException: SQL validation failed. SQL validation failed. null at org.apache.flink.table.planner.calcite.FlinkPlannerImpl.org$apache$flink$table$planner$calcite$FlinkPlannerImpl$$validate(FlinkPlannerImpl.scala:189) at org.apache.flink.table.planner.calcite.FlinkPlannerImpl.validate(FlinkPlannerImpl.scala:115) at org.apache.flink.table.planner.operations.SqlToOperationConverter.convert(SqlToOperationConverter.java:282) at org.apache.flink.table.planner.delegation.ParserImpl.parse(ParserImpl.java:106) at org.apache.flink.table.api.internal.TableEnvironmentImpl.explainSql(TableEnvironmentImpl.java:696) at org.apache.flink.table.api.TableEnvironment.explainSql(TableEnvironment.java:976) at org.apache.flink.table.planner.plan.stream.sql.ColumnExpansionTest.testExplicitTableWithinTableFunctionWithInsertIntoNamedColumns(ColumnExpansionTest.java:279) at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.base/java.lang.reflect.Method.invoke(Method.java:566) at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59) at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56) at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17) at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26) at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) at org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100) at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:366) at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103) at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63) at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331) at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79) at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329) at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66) at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293) at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306) at org.junit.runners.ParentRunner.run(ParentRunner.java:413) at org.junit.runner.JUnitCore.run(JUnitCore.java:137) at com.intellij.junit4.JUnit4IdeaTestRunner.startRunnerWithArgs(JUnit4IdeaTestRunner.java:69) at com.intellij.rt.junit.IdeaTestRunner$Repeater$1.execute(IdeaTestRunner.java:38) at com.intellij.rt.execution.junit.TestsRepeater.repeat(TestsRepeater.java:11) at com.intellij.rt.junit.IdeaTestRunner$Repeater.startRunnerWithArgs(IdeaTestRunner.java:35) at com.intellij.rt.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:232) at com.intellij.rt.junit.JUnitStarter.main(JUnitStarter.java:55) Caused by: 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:189) at org.apache.flink.table.planner.calcite.FlinkPlannerImpl.validate(FlinkPlannerImpl.scala:115) at org.apache.flink.table.planner.calcite.FlinkPlannerImpl.validateRichSqlInsert(FlinkPlannerImpl.scala:294) at org.apache.flink.table.planner.calcite.FlinkPlannerImpl.org$apache$flink$table$planner$calcite$FlinkPlannerImpl$$validate(FlinkPlannerImpl.scala:177) ... 34 more Caused by: java.lang.NullPointerException at org.apache.flink.table.planner.calcite.FlinkCalciteSqlValidator.performUnconditionalRewrites(FlinkCalciteSqlValidator.java:213) at org.apache.calcite.sql.validate.SqlValidatorImpl.performUnconditionalRewrites(SqlValidatorImpl.java:1288) at org.apache.flink.table.planner.calcite.FlinkCalciteSqlValidator.performUnconditionalRewrites(FlinkCalciteSqlValidator.java:204) at org.apache.calcite.sql.validate.SqlValidatorImpl.performUnconditionalRewrites(SqlValidatorImpl.java:1288) at org.apache.flink.table.planner.calcite.FlinkCalciteSqlValidator.performUnconditionalRewrites(FlinkCalciteSqlValidator.java:204) at org.apache.calcite.sql.validate.SqlValidatorImpl.performUnconditionalRewrites(SqlValidatorImpl.java:1288) at org.apache.flink.table.planner.calcite.FlinkCalciteSqlValidator.performUnconditionalRewrites(FlinkCalciteSqlValidator.java:204) at org.apache.calcite.sql.validate.SqlValidatorImpl.validateScopedExpression(SqlValidatorImpl.java:985) at org.apache.calcite.sql.validate.SqlValidatorImpl.validate(SqlValidatorImpl.java:741) at org.apache.flink.table.planner.calcite.FlinkPlannerImpl.org$apache$flink$table$planner$calcite$FlinkPlannerImpl$$validate(FlinkPlannerImpl.scala:185) ... 37 more
Attachments
Issue Links
- links to