Details
-
Bug
-
Status: In Progress
-
Major
-
Resolution: Unresolved
-
1.14.3
-
None
Description
The following test case insert specific columns with sql hint for sink table:
@Test def testPartialInsertWithReorderAndHint(): Unit = { tEnv.executeSql( s""" |CREATE TABLE default_database.testSink ( | `a` INT, | `b` AS `a` + 1, | `c` STRING, | `d` INT, | `e` DOUBLE |) |PARTITIONED BY (`c`, `d`) |WITH ( | 'connector' = 'values', | 'sink-insert-only' = 'true' |) |""".stripMargin) registerCollection("MyTable", simpleData2, simpleType2, "x, y", nullableOfSimpleData2) tEnv.executeSql( s""" |INSERT INTO default_database.testSink /*+ OPTIONS('sink-insert-only'='false') */ (e, d, c) |SELECT sum(y), 1, '2021' FROM MyTable GROUP BY x |""".stripMargin).await() val expected = List( "null,2021,1,0.1", "null,2021,1,0.4", "null,2021,1,1.0", "null,2021,1,2.2", "null,2021,1,3.9") val result = TestValuesTableFactory.getResults("testSink") assertEquals(expected.sorted, result.sorted) }
Fails with following exception stack
org.apache.flink.table.api.SqlParserException: SQL parse failed. Non-query expression encountered in illegal context at org.apache.flink.table.planner.calcite.CalciteParser.parse(CalciteParser.java:56) at org.apache.flink.table.planner.delegation.ParserImpl.parse(ParserImpl.java:74) at org.apache.flink.table.api.internal.TableEnvironmentImpl.executeSql(TableEnvironmentImpl.java:660) at org.apache.flink.table.planner.runtime.batch.table.TableSinkITCase.testPartialInsertWithDynamicAndStaticPartition1(TableSinkITCase.scala:534) at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:498) at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:50) at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:47) 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.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27) at org.junit.rules.ExpectedException$ExpectedExceptionStatement.evaluate(ExpectedException.java:239) at org.junit.rules.TestWatcher$1.evaluate(TestWatcher.java:55) at org.junit.rules.RunRules.evaluate(RunRules.java:20) at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:325) at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:78) at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:57) at org.junit.runners.ParentRunner$3.run(ParentRunner.java:290) at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:71) at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:288) at org.junit.runners.ParentRunner.access$000(ParentRunner.java:58) at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:268) at org.junit.rules.ExternalResource$1.evaluate(ExternalResource.java:48) at org.junit.rules.ExternalResource$1.evaluate(ExternalResource.java:48) at org.junit.rules.RunRules.evaluate(RunRules.java:20) at org.junit.runners.ParentRunner.run(ParentRunner.java:363) 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.startRunnerWithArgs(IdeaTestRunner.java:33) at com.intellij.rt.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:235) at com.intellij.rt.junit.JUnitStarter.main(JUnitStarter.java:54) Caused by: org.apache.calcite.sql.parser.SqlParseException: Non-query expression encountered in illegal context at org.apache.flink.sql.parser.impl.FlinkSqlParserImpl.convertException(FlinkSqlParserImpl.java:442) at org.apache.flink.sql.parser.impl.FlinkSqlParserImpl.normalizeException(FlinkSqlParserImpl.java:205) at org.apache.calcite.sql.parser.SqlParser.handleException(SqlParser.java:140) at org.apache.calcite.sql.parser.SqlParser.parseQuery(SqlParser.java:155) at org.apache.calcite.sql.parser.SqlParser.parseStmt(SqlParser.java:180) at org.apache.flink.table.planner.calcite.CalciteParser.parse(CalciteParser.java:54) ... 33 more Caused by: org.apache.calcite.runtime.CalciteException: Non-query expression encountered in illegal context at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method) at sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:62) at sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45) at java.lang.reflect.Constructor.newInstance(Constructor.java:423) 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:365) at org.apache.flink.sql.parser.impl.FlinkSqlParserImpl.Expression3(FlinkSqlParserImpl.java:19011) at org.apache.flink.sql.parser.impl.FlinkSqlParserImpl.Expression2b(FlinkSqlParserImpl.java:18680) at org.apache.flink.sql.parser.impl.FlinkSqlParserImpl.Expression2(FlinkSqlParserImpl.java:18721) at org.apache.flink.sql.parser.impl.FlinkSqlParserImpl.Expression(FlinkSqlParserImpl.java:18652) at org.apache.flink.sql.parser.impl.FlinkSqlParserImpl.LeafQueryOrExpr(FlinkSqlParserImpl.java:18629) at org.apache.flink.sql.parser.impl.FlinkSqlParserImpl.QueryOrExpr(FlinkSqlParserImpl.java:18089) at org.apache.flink.sql.parser.impl.FlinkSqlParserImpl.OrderedQueryOrExpr(FlinkSqlParserImpl.java:558) at org.apache.flink.sql.parser.impl.FlinkSqlParserImpl.ParenthesizedQueryOrCommaList(FlinkSqlParserImpl.java:766) at org.apache.flink.sql.parser.impl.FlinkSqlParserImpl.Expression3(FlinkSqlParserImpl.java:19046) at org.apache.flink.sql.parser.impl.FlinkSqlParserImpl.Expression2b(FlinkSqlParserImpl.java:18680) at org.apache.flink.sql.parser.impl.FlinkSqlParserImpl.Expression2(FlinkSqlParserImpl.java:18721) at org.apache.flink.sql.parser.impl.FlinkSqlParserImpl.Expression(FlinkSqlParserImpl.java:18652) at org.apache.flink.sql.parser.impl.FlinkSqlParserImpl.LeafQueryOrExpr(FlinkSqlParserImpl.java:18629) at org.apache.flink.sql.parser.impl.FlinkSqlParserImpl.QueryOrExpr(FlinkSqlParserImpl.java:18089) at org.apache.flink.sql.parser.impl.FlinkSqlParserImpl.OrderedQueryOrExpr(FlinkSqlParserImpl.java:558) at org.apache.flink.sql.parser.impl.FlinkSqlParserImpl.RichSqlInsert(FlinkSqlParserImpl.java:5709) at org.apache.flink.sql.parser.impl.FlinkSqlParserImpl.SqlStmt(FlinkSqlParserImpl.java:3342) at org.apache.flink.sql.parser.impl.FlinkSqlParserImpl.SqlStmtEof(FlinkSqlParserImpl.java:3882) at org.apache.flink.sql.parser.impl.FlinkSqlParserImpl.parseSqlStmtEof(FlinkSqlParserImpl.java:253) at org.apache.calcite.sql.parser.SqlParser.parseQuery(SqlParser.java:153) ... 35 moreDisconnected from the target VM, address: '127.0.0.1:62255', transport: 'socket'Process finished with exit code -1
And insert into specific partition with sql hint also fails.
Attachments
Issue Links
- duplicates
-
FLINK-27683 Insert into (column1, column2) Values(.....) fails with SQL hints
- Closed
- links to