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

Flink SQL Error while applying rule AggregateReduceGroupingRule

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Closed
    • Critical
    • Resolution: Fixed
    • 1.12.3, 1.12.4
    • 1.12.5
    • Table SQL / Planner
    • None

    Description

      When I updated version from 1.12.1 to 1.12.4, the follow SQL was cannot run.

      //代码占位符
      String retailSql = "SELECT\n" +
              "    customer_id,\n" +
              "    ware_virtual_category,\n" +
              "    min(pay_datetime) as pay_datetime\n" +
              " FROM " +
              "   `kudu`.`default_database`.`impala::cube_kudu.dwd_order_retail_order_pay` \n" +
              " WHERE " +
              "   pay_date = TO_TIMESTAMP('" + partitionTime + "')" +
              " AND " +
              "   freight_flag in (0)  " + 
              " AND   " +
              "   order_pay_type <> '3' " + 
              " GROUP BY \n" +
              "    customer_id," +
              "    ware_virtual_category";

       

      the error message is follow:

      //代码占位符
      Exception in thread "main" java.lang.RuntimeException: Error while applying rule AggregateReduceGroupingRule, args [rel#833:FlinkLogicalAggregate.LOGICAL.any.[](input=RelSubset#832,group={0, 1},pay_datetime=MIN($2))]Exception in thread "main" java.lang.RuntimeException: Error while applying rule AggregateReduceGroupingRule, args [rel#833:FlinkLogicalAggregate.LOGICAL.any.[](input=RelSubset#832,group={0, 1},pay_datetime=MIN($2))] at org.apache.calcite.plan.volcano.VolcanoRuleCall.onMatch(VolcanoRuleCall.java:256) at org.apache.calcite.plan.volcano.IterativeRuleDriver.drive(IterativeRuleDriver.java:58) at org.apache.calcite.plan.volcano.VolcanoPlanner.findBestExp(VolcanoPlanner.java:510) at org.apache.calcite.tools.Programs$RuleSetProgram.run(Programs.java:312) at org.apache.flink.table.planner.plan.optimize.program.FlinkVolcanoProgram.optimize(FlinkVolcanoProgram.scala:64) at org.apache.flink.table.planner.plan.optimize.program.FlinkChainedProgram$$anonfun$optimize$1.apply(FlinkChainedProgram.scala:62) at org.apache.flink.table.planner.plan.optimize.program.FlinkChainedProgram$$anonfun$optimize$1.apply(FlinkChainedProgram.scala:58) at scala.collection.TraversableOnce$$anonfun$foldLeft$1.apply(TraversableOnce.scala:157) at scala.collection.TraversableOnce$$anonfun$foldLeft$1.apply(TraversableOnce.scala:157) at scala.collection.Iterator$class.foreach(Iterator.scala:891) at scala.collection.AbstractIterator.foreach(Iterator.scala:1334) at scala.collection.IterableLike$class.foreach(IterableLike.scala:72) at scala.collection.AbstractIterable.foreach(Iterable.scala:54) at scala.collection.TraversableOnce$class.foldLeft(TraversableOnce.scala:157) at scala.collection.AbstractTraversable.foldLeft(Traversable.scala:104) at org.apache.flink.table.planner.plan.optimize.program.FlinkChainedProgram.optimize(FlinkChainedProgram.scala:57) at org.apache.flink.table.planner.plan.optimize.BatchCommonSubGraphBasedOptimizer.optimizeTree(BatchCommonSubGraphBasedOptimizer.scala:86) at org.apache.flink.table.planner.plan.optimize.BatchCommonSubGraphBasedOptimizer.org$apache$flink$table$planner$plan$optimize$BatchCommonSubGraphBasedOptimizer$$optimizeBlock(BatchCommonSubGraphBasedOptimizer.scala:57) at org.apache.flink.table.planner.plan.optimize.BatchCommonSubGraphBasedOptimizer$$anonfun$doOptimize$1.apply(BatchCommonSubGraphBasedOptimizer.scala:45) at org.apache.flink.table.planner.plan.optimize.BatchCommonSubGraphBasedOptimizer$$anonfun$doOptimize$1.apply(BatchCommonSubGraphBasedOptimizer.scala:45) at scala.collection.immutable.List.foreach(List.scala:392) at org.apache.flink.table.planner.plan.optimize.BatchCommonSubGraphBasedOptimizer.doOptimize(BatchCommonSubGraphBasedOptimizer.scala:45) at org.apache.flink.table.planner.plan.optimize.CommonSubGraphBasedOptimizer.optimize(CommonSubGraphBasedOptimizer.scala:77) at org.apache.flink.table.planner.delegation.PlannerBase.optimize(PlannerBase.scala:287) at org.apache.flink.table.planner.delegation.PlannerBase.translate(PlannerBase.scala:160) at org.apache.flink.table.api.internal.TableEnvironmentImpl.translate(TableEnvironmentImpl.java:1329) at org.apache.flink.table.api.internal.TableEnvironmentImpl.executeInternal(TableEnvironmentImpl.java:676) at org.apache.flink.table.api.internal.StatementSetImpl.execute(StatementSetImpl.java:98) at com.kad.cube.dws.day.DwsAllWareCategoryCustomerPayTimeDay.sinkToKudu(DwsAllWareCategoryCustomerPayTimeDay.java:54) at com.kad.cube.dws.day.DwsAllWareCategoryCustomerPayTimeDay.main(DwsAllWareCategoryCustomerPayTimeDay.java:45)Caused by: java.lang.IllegalArgumentException at org.apache.flink.util.Preconditions.checkArgument(Preconditions.java:122) at org.apache.flink.table.planner.plan.metadata.FlinkRelMetadataQuery.getUniqueGroups(FlinkRelMetadataQuery.java:197) at org.apache.flink.table.planner.plan.metadata.FlinkRelMdUniqueGroups.getUniqueGroups(FlinkRelMdUniqueGroups.scala:411) at GeneratedMetadataHandler_UniqueGroups.getUniqueGroups_$(Unknown Source) at GeneratedMetadataHandler_UniqueGroups.getUniqueGroups(Unknown Source) at org.apache.flink.table.planner.plan.metadata.FlinkRelMetadataQuery.getUniqueGroups(FlinkRelMetadataQuery.java:196) at org.apache.flink.table.planner.plan.metadata.FlinkRelMdUniqueGroups.getUniqueGroupsOfProject(FlinkRelMdUniqueGroups.scala:131) at org.apache.flink.table.planner.plan.metadata.FlinkRelMdUniqueGroups.getUniqueGroups(FlinkRelMdUniqueGroups.scala:92) at GeneratedMetadataHandler_UniqueGroups.getUniqueGroups_$(Unknown Source) at GeneratedMetadataHandler_UniqueGroups.getUniqueGroups(Unknown Source) at org.apache.flink.table.planner.plan.metadata.FlinkRelMetadataQuery.getUniqueGroups(FlinkRelMetadataQuery.java:196) at org.apache.flink.table.planner.plan.metadata.FlinkRelMdUniqueGroups.getUniqueGroups(FlinkRelMdUniqueGroups.scala:411) at GeneratedMetadataHandler_UniqueGroups.getUniqueGroups_$(Unknown Source) at GeneratedMetadataHandler_UniqueGroups.getUniqueGroups(Unknown Source) at org.apache.flink.table.planner.plan.metadata.FlinkRelMetadataQuery.getUniqueGroups(FlinkRelMetadataQuery.java:196) at org.apache.flink.table.planner.plan.metadata.FlinkRelMdUniqueGroups.getUniqueGroupsOfProject(FlinkRelMdUniqueGroups.scala:131) at org.apache.flink.table.planner.plan.metadata.FlinkRelMdUniqueGroups.getUniqueGroups(FlinkRelMdUniqueGroups.scala:92) at GeneratedMetadataHandler_UniqueGroups.getUniqueGroups_$(Unknown Source) at GeneratedMetadataHandler_UniqueGroups.getUniqueGroups(Unknown Source) at org.apache.flink.table.planner.plan.metadata.FlinkRelMetadataQuery.getUniqueGroups(FlinkRelMetadataQuery.java:196) at org.apache.flink.table.planner.plan.metadata.FlinkRelMdUniqueGroups.getUniqueGroups(FlinkRelMdUniqueGroups.scala:411) at GeneratedMetadataHandler_UniqueGroups.getUniqueGroups_$(Unknown Source) at GeneratedMetadataHandler_UniqueGroups.getUniqueGroups(Unknown Source) at org.apache.flink.table.planner.plan.metadata.FlinkRelMetadataQuery.getUniqueGroups(FlinkRelMetadataQuery.java:196) at org.apache.flink.table.planner.plan.rules.logical.AggregateReduceGroupingRule.onMatch(AggregateReduceGroupingRule.scala:56) at org.apache.calcite.plan.volcano.VolcanoRuleCall.onMatch(VolcanoRuleCall.java:229) ... 29 more

       

      rel#208:FlinkLogicalTableSourceScan.LOGICAL.any.[](table=[kudu, default_database, impala::cube_kudu.dwd_order_retail_order_pay, filter=[equals(pay_date, 2021-06-23T00:00)], project=[customer_id, order_source_id, order_sale_channel_id, pay_datetime, order_pay_type]],fields=customer_id, order_source_id, order_sale_channel_id, pay_datetime, order_pay_type)

       

      Compare to version 1.12.1 :

      Attachments

        1. image-2021-06-24-18-20-54-752.png
          92 kB
          zhengjiewen
        2. image-2021-06-24-18-13-16-056.png
          103 kB
          zhengjiewen
        3. image-2021-06-24-18-04-03-473.png
          72 kB
          zhengjiewen
        4. yarn.txt
          88 kB
          zhengjiewen

        Issue Links

          Activity

            People

              godfreyhe godfrey he
              zhengjiewen zhengjiewen
              Votes:
              0 Vote for this issue
              Watchers:
              7 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: