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

SQL parser fails to parse group by statements with brackets in views

    XMLWordPrintableJSON

Details

    Description

      Add the following case to org.apache.flink.table.planner.runtime.batch.sql.CalcITCase to reproduce this bug.

      def myTest(): Unit = {
        tEnv.executeSql("CREATE VIEW my_view AS SELECT max(a) AS mx FROM Table3 group by (b, c)")
        checkResult(
          "SELECT mx FROM my_view WHERE mx > 5",
          Seq())
      }
      

      The exception stack is

      org.apache.flink.table.api.SqlParserException: SQL parse failed. Encountered "." at line 3, column 22.
      Was expecting one of:
          ")" ...
          "," ...
          
      
      	at org.apache.flink.table.planner.parse.CalciteParser.parse(CalciteParser.java:56)
      	at org.apache.flink.table.planner.calcite.FlinkPlannerImpl$ToRelContextImpl.expandView(FlinkPlannerImpl.scala:263)
      	at org.apache.calcite.plan.ViewExpanders$1.expandView(ViewExpanders.java:52)
      	at org.apache.flink.table.planner.catalog.SqlCatalogViewTable.convertToRel(SqlCatalogViewTable.java:63)
      	at org.apache.flink.table.planner.plan.schema.ExpandingPreparingTable.expand(ExpandingPreparingTable.java:68)
      	at org.apache.flink.table.planner.plan.schema.ExpandingPreparingTable.toRel(ExpandingPreparingTable.java:64)
      	at org.apache.calcite.sql2rel.SqlToRelConverter.toRel(SqlToRelConverter.java:3592)
      	at org.apache.calcite.sql2rel.SqlToRelConverter.convertIdentifier(SqlToRelConverter.java:2514)
      	at org.apache.calcite.sql2rel.SqlToRelConverter.convertFrom(SqlToRelConverter.java:2151)
      	at org.apache.calcite.sql2rel.SqlToRelConverter.convertFrom(SqlToRelConverter.java:2100)
      	at org.apache.calcite.sql2rel.SqlToRelConverter.convertFrom(SqlToRelConverter.java:2057)
      	at org.apache.calcite.sql2rel.SqlToRelConverter.convertSelectImpl(SqlToRelConverter.java:663)
      	at org.apache.calcite.sql2rel.SqlToRelConverter.convertSelect(SqlToRelConverter.java:644)
      	at org.apache.calcite.sql2rel.SqlToRelConverter.convertQueryRecursive(SqlToRelConverter.java:3445)
      	at org.apache.calcite.sql2rel.SqlToRelConverter.convertQuery(SqlToRelConverter.java:570)
      	at org.apache.flink.table.planner.calcite.FlinkPlannerImpl.org$apache$flink$table$planner$calcite$FlinkPlannerImpl$$rel(FlinkPlannerImpl.scala:178)
      	at org.apache.flink.table.planner.calcite.FlinkPlannerImpl.rel(FlinkPlannerImpl.scala:170)
      	at org.apache.flink.table.planner.operations.SqlToOperationConverter.toQueryOperation(SqlToOperationConverter.java:975)
      	at org.apache.flink.table.planner.operations.SqlToOperationConverter.convertSqlQuery(SqlToOperationConverter.java:944)
      	at org.apache.flink.table.planner.operations.SqlToOperationConverter.convert(SqlToOperationConverter.java:279)
      	at org.apache.flink.table.planner.delegation.ParserImpl.parse(ParserImpl.java:99)
      	at org.apache.flink.table.api.internal.TableEnvironmentImpl.sqlQuery(TableEnvironmentImpl.java:703)
      	at org.apache.flink.table.planner.runtime.utils.BatchTestBase.parseQuery(BatchTestBase.scala:297)
      	at org.apache.flink.table.planner.runtime.utils.BatchTestBase.check(BatchTestBase.scala:139)
      	at org.apache.flink.table.planner.runtime.utils.BatchTestBase.checkResult(BatchTestBase.scala:106)
      	at org.apache.flink.table.planner.runtime.batch.sql.CalcITCase.myTest(CalcITCase.scala:70)
      	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.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.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: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 "." at line 3, column 22.
      Was expecting one of:
          ")" ...
          "," ...
          
      	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)
      	... 55 more
      Caused by: org.apache.flink.sql.parser.impl.ParseException: Encountered "." at line 3, column 22.
      Was expecting one of:
          ")" ...
          "," ...
          
      	at org.apache.flink.sql.parser.impl.FlinkSqlParserImpl.generateParseException(FlinkSqlParserImpl.java:40575)
      	at org.apache.flink.sql.parser.impl.FlinkSqlParserImpl.jj_consume_token(FlinkSqlParserImpl.java:40386)
      	at org.apache.flink.sql.parser.impl.FlinkSqlParserImpl.ParenthesizedSimpleIdentifierList(FlinkSqlParserImpl.java:25054)
      	at org.apache.flink.sql.parser.impl.FlinkSqlParserImpl.Expression3(FlinkSqlParserImpl.java:19813)
      	at org.apache.flink.sql.parser.impl.FlinkSqlParserImpl.Expression2b(FlinkSqlParserImpl.java:19469)
      	at org.apache.flink.sql.parser.impl.FlinkSqlParserImpl.Expression2(FlinkSqlParserImpl.java:19510)
      	at org.apache.flink.sql.parser.impl.FlinkSqlParserImpl.Expression(FlinkSqlParserImpl.java:19441)
      	at org.apache.flink.sql.parser.impl.FlinkSqlParserImpl.GroupingElement(FlinkSqlParserImpl.java:14865)
      	at org.apache.flink.sql.parser.impl.FlinkSqlParserImpl.GroupingElementList(FlinkSqlParserImpl.java:14352)
      	at org.apache.flink.sql.parser.impl.FlinkSqlParserImpl.GroupByOpt(FlinkSqlParserImpl.java:14339)
      	at org.apache.flink.sql.parser.impl.FlinkSqlParserImpl.SqlSelect(FlinkSqlParserImpl.java:7863)
      	at org.apache.flink.sql.parser.impl.FlinkSqlParserImpl.LeafQuery(FlinkSqlParserImpl.java:693)
      	at org.apache.flink.sql.parser.impl.FlinkSqlParserImpl.LeafQueryOrExpr(FlinkSqlParserImpl.java:19424)
      	at org.apache.flink.sql.parser.impl.FlinkSqlParserImpl.QueryOrExpr(FlinkSqlParserImpl.java:18876)
      	at org.apache.flink.sql.parser.impl.FlinkSqlParserImpl.OrderedQueryOrExpr(FlinkSqlParserImpl.java:567)
      	at org.apache.flink.sql.parser.impl.FlinkSqlParserImpl.SqlStmt(FlinkSqlParserImpl.java:3882)
      	at org.apache.flink.sql.parser.impl.FlinkSqlParserImpl.SqlStmtEof(FlinkSqlParserImpl.java:3921)
      	at org.apache.flink.sql.parser.impl.FlinkSqlParserImpl.parseSqlStmtEof(FlinkSqlParserImpl.java:262)
      	at org.apache.calcite.sql.parser.SqlParser.parseQuery(SqlParser.java:153)
      	... 57 more
      

      This is because views in queries will be expanded. After expanding the query becomes

      SELECT MAX(`Table3`.`a`) AS `mx`
      FROM `default_catalog`.`default_database`.`Table3`
      GROUP BY ROW(`Table3`.`b`, `Table3`.`c`)
      

      SQL parser fails to parse complete column names in ROW.

      CALCITE-4456 seems to fix this issue, however it is in Calcite 1.27

      Attachments

        Issue Links

          Activity

            People

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

              Dates

                Created:
                Updated:
                Resolved: