Uploaded image for project: 'Flink'
  1. Flink
  2. FLINK-22450

SQL parser fails to parse subquery containing INTERSECT in a view

    XMLWordPrintableJSON

Details

    Description

      Add the following test case to TableEnvironmentITCase to reproduce this bug.

      @Test
      def myTest(): Unit = {
        tEnv.executeSql(
          """
            |CREATE TABLE T1 (
            |  a INT,
            |  b BIGINT
            |) WITH (
            |  'connector'='values'
            |)
            |""".stripMargin)
        tEnv.executeSql(
          """
            |CREATE TABLE T2 (
            |  c INT,
            |  d BIGINT
            |) WITH (
            |  'connector'='values'
            |)
            |""".stripMargin)
        tEnv.executeSql(
          """
            |CREATE TABLE T3 (
            |  c INT,
            |  d BIGINT
            |) WITH (
            |  'connector'='values'
            |)
            |""".stripMargin)
        tEnv.executeSql("CREATE VIEW myView AS SELECT * FROM T1, (SELECT * FROM T2 WHERE c > 0 INTERSECT SELECT * FROM T3 WHERE c > 0) WHERE a = c")
        System.out.println(tEnv.explainSql("SELECT * FROM myView"))
      }
      

      The exception stack is

      org.apache.flink.table.api.SqlParserException: SQL parse failed. Encountered ", SELECT" at line 2, column 10.
      Was expecting one of:
          <EOF> 
          "AS" ...
          "EXCEPT" ...
          "EXTEND" ...
          "FETCH" ...
          "FOR" ...
          "GROUP" ...
          "HAVING" ...
          "INTERSECT" ...
          "LIMIT" ...
          "MATCH_RECOGNIZE" ...
          "OFFSET" ...
          "ORDER" ...
          "PIVOT" ...
          "MINUS" ...
          "TABLESAMPLE" ...
          "UNION" ...
          "WHERE" ...
          "WINDOW" ...
          "(" ...
          <BRACKET_QUOTED_IDENTIFIER> ...
          <QUOTED_IDENTIFIER> ...
          <BACK_QUOTED_IDENTIFIER> ...
          <HYPHENATED_IDENTIFIER> ...
          <IDENTIFIER> ...
          <UNICODE_QUOTED_IDENTIFIER> ...
          "/*+" ...
          "NATURAL" ...
          "JOIN" ...
          "INNER" ...
          "LEFT" ...
          "RIGHT" ...
          "FULL" ...
          "CROSS" ...
          "," <IDENTIFIER> ...
          "," <HYPHENATED_IDENTIFIER> ...
          "," <QUOTED_IDENTIFIER> ...
          "," <BACK_QUOTED_IDENTIFIER> ...
          "," <BRACKET_QUOTED_IDENTIFIER> ...
          "," <UNICODE_QUOTED_IDENTIFIER> ...
          "," "LATERAL" ...
          "," "(" ...
          "," "UNNEST" ...
          "," "TABLE" ...
          "OUTER" ...
          "." ...
          
      
      	at org.apache.flink.table.planner.parse.CalciteParser.parse(CalciteParser.java:56)
      	at org.apache.flink.table.planner.utils.Expander.expanded(Expander.java:83)
      	at org.apache.flink.table.planner.operations.SqlToOperationConverter.convertViewQuery(SqlToOperationConverter.java:849)
      	at org.apache.flink.table.planner.operations.SqlToOperationConverter.convertCreateView(SqlToOperationConverter.java:819)
      	at org.apache.flink.table.planner.operations.SqlToOperationConverter.convert(SqlToOperationConverter.java:248)
      	at org.apache.flink.table.planner.delegation.ParserImpl.parse(ParserImpl.java:99)
      	at org.apache.flink.table.api.internal.TableEnvironmentImpl.executeSql(TableEnvironmentImpl.java:723)
      	at org.apache.flink.table.api.TableEnvironmentITCase.myTest(TableEnvironmentITCase.scala:116)
      	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.rules.ExpectedException$ExpectedExceptionStatement.evaluate(ExpectedException.java:239)
      	at org.junit.rules.ExternalResource$1.evaluate(ExternalResource.java:48)
      	at org.apache.flink.util.TestNameProvider$1.evaluate(TestNameProvider.java:45)
      	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.runners.ParentRunner.run(ParentRunner.java:363)
      	at org.junit.runners.Suite.runChild(Suite.java:128)
      	at org.junit.runners.Suite.runChild(Suite.java:27)
      	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.runners.ParentRunner.run(ParentRunner.java:363)
      	at org.junit.runner.JUnitCore.run(JUnitCore.java:137)
      	at com.intellij.junit4.JUnit4IdeaTestRunner.startRunnerWithArgs(JUnit4IdeaTestRunner.java:68)
      	at com.intellij.rt.junit.IdeaTestRunner$Repeater.startRunnerWithArgs(IdeaTestRunner.java:33)
      	at com.intellij.rt.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:230)
      	at com.intellij.rt.junit.JUnitStarter.main(JUnitStarter.java:58)
      Caused by: org.apache.calcite.sql.parser.SqlParseException: Encountered ", SELECT" at line 2, column 10.
      Was expecting one of:
          <EOF> 
          "AS" ...
          "EXCEPT" ...
          "EXTEND" ...
          "FETCH" ...
          "FOR" ...
          "GROUP" ...
          "HAVING" ...
          "INTERSECT" ...
          "LIMIT" ...
          "MATCH_RECOGNIZE" ...
          "OFFSET" ...
          "ORDER" ...
          "PIVOT" ...
          "MINUS" ...
          "TABLESAMPLE" ...
          "UNION" ...
          "WHERE" ...
          "WINDOW" ...
          "(" ...
          <BRACKET_QUOTED_IDENTIFIER> ...
          <QUOTED_IDENTIFIER> ...
          <BACK_QUOTED_IDENTIFIER> ...
          <HYPHENATED_IDENTIFIER> ...
          <IDENTIFIER> ...
          <UNICODE_QUOTED_IDENTIFIER> ...
          "/*+" ...
          "NATURAL" ...
          "JOIN" ...
          "INNER" ...
          "LEFT" ...
          "RIGHT" ...
          "FULL" ...
          "CROSS" ...
          "," <IDENTIFIER> ...
          "," <HYPHENATED_IDENTIFIER> ...
          "," <QUOTED_IDENTIFIER> ...
          "," <BACK_QUOTED_IDENTIFIER> ...
          "," <BRACKET_QUOTED_IDENTIFIER> ...
          "," <UNICODE_QUOTED_IDENTIFIER> ...
          "," "LATERAL" ...
          "," "(" ...
          "," "UNNEST" ...
          "," "TABLE" ...
          "OUTER" ...
          "." ...
          
      	at org.apache.flink.sql.parser.impl.FlinkSqlParserImpl.convertException(FlinkSqlParserImpl.java:451)
      	at org.apache.flink.sql.parser.impl.FlinkSqlParserImpl.normalizeException(FlinkSqlParserImpl.java:214)
      	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.parse.CalciteParser.parse(CalciteParser.java:54)
      	... 43 more
      Caused by: org.apache.flink.sql.parser.impl.ParseException: Encountered ", SELECT" at line 2, column 10.
      Was expecting one of:
          <EOF> 
          "AS" ...
          "EXCEPT" ...
          "EXTEND" ...
          "FETCH" ...
          "FOR" ...
          "GROUP" ...
          "HAVING" ...
          "INTERSECT" ...
          "LIMIT" ...
          "MATCH_RECOGNIZE" ...
          "OFFSET" ...
          "ORDER" ...
          "PIVOT" ...
          "MINUS" ...
          "TABLESAMPLE" ...
          "UNION" ...
          "WHERE" ...
          "WINDOW" ...
          "(" ...
          <BRACKET_QUOTED_IDENTIFIER> ...
          <QUOTED_IDENTIFIER> ...
          <BACK_QUOTED_IDENTIFIER> ...
          <HYPHENATED_IDENTIFIER> ...
          <IDENTIFIER> ...
          <UNICODE_QUOTED_IDENTIFIER> ...
          "/*+" ...
          "NATURAL" ...
          "JOIN" ...
          "INNER" ...
          "LEFT" ...
          "RIGHT" ...
          "FULL" ...
          "CROSS" ...
          "," <IDENTIFIER> ...
          "," <HYPHENATED_IDENTIFIER> ...
          "," <QUOTED_IDENTIFIER> ...
          "," <BACK_QUOTED_IDENTIFIER> ...
          "," <BRACKET_QUOTED_IDENTIFIER> ...
          "," <UNICODE_QUOTED_IDENTIFIER> ...
          "," "LATERAL" ...
          "," "(" ...
          "," "UNNEST" ...
          "," "TABLE" ...
          "OUTER" ...
          "." ...
          
      	at org.apache.flink.sql.parser.impl.FlinkSqlParserImpl.generateParseException(FlinkSqlParserImpl.java:40579)
      	at org.apache.flink.sql.parser.impl.FlinkSqlParserImpl.jj_consume_token(FlinkSqlParserImpl.java:40390)
      	at org.apache.flink.sql.parser.impl.FlinkSqlParserImpl.SqlStmtEof(FlinkSqlParserImpl.java:3922)
      	at org.apache.flink.sql.parser.impl.FlinkSqlParserImpl.parseSqlStmtEof(FlinkSqlParserImpl.java:262)
      	at org.apache.calcite.sql.parser.SqlParser.parseQuery(SqlParser.java:153)
      	... 45 more
      

      This is because when expanding views, we use SqlNode#toString to parse a SqlNode to its corresponding SQL string (See SqlToOperationConverter#convertViewQuery, notice the statement String originalQuery = getQuotedSqlString(query); and the following usages of originalQuery). However the result of SqlNode#toString might not be a valid SQL string. For example

      CREATE VIEW myView AS SELECT * FROM T1, (SELECT * FROM T2 WHERE c > 0 INTERSECT SELECT * FROM T3 WHERE c > 0) WHERE a = c
      

      The result of SqlNode#toString for the SqlNode above is

      CREATE VIEW myView AS SELECT * FROM T1, SELECT * FROM T2 WHERE c > 0 INTERSECT SELECT * FROM T3 WHERE c > 0 WHERE a = c
      

      Notice the lacking of brackets around the subquery.

      This is because INTERSECT is an operator with high precedence, so when unparsing the SqlNode, Calcite thinks that it is not necessary to add brackets around the subquery.

      Attachments

        Activity

          People

            Unassigned Unassigned
            TsReaper Caizhi Weng
            Votes:
            0 Vote for this issue
            Watchers:
            5 Start watching this issue

            Dates

              Created:
              Updated: