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

Fail to generate code for SearchOperator

    XMLWordPrintableJSON

Details

    Description

      Can be reproduced with the following code with Hive dialect

      // hive dialect
      
      tableEnv.executeSql("create table table1 (id int, val string, val1 string, dimid int)");
      tableEnv.executeSql("create table table3 (id int)");
      
      CollectionUtil.iteratorToList(
              tableEnv.executeSql(
                              "select table1.id, table1.val, table1.val1 from table1 left semi join"
                                      + " table3 on table1.dimid = table3.id and table3.id = 100 where table1.dimid = 200")
                      .collect());

      The  plan is 

      LogicalSink(table=[*anonymous_collect$1*], fields=[id, val, val1])
        LogicalProject(id=[$0], val=[$1], val1=[$2])
          LogicalFilter(condition=[=($3, 200)])
            LogicalJoin(condition=[AND(=($3, $4), =($4, 100))], joinType=[semi])
              LogicalTableScan(table=[[test-catalog, default, table1]])
              LogicalTableScan(table=[[test-catalog, default, table3]])BatchPhysicalSink(table=[*anonymous_collect$1*], fields=[id, val, val1])
        BatchPhysicalNestedLoopJoin(joinType=[LeftSemiJoin], where=[$f1], select=[id, val, val1], build=[right])
          BatchPhysicalCalc(select=[id, val, val1], where=[=(dimid, 200)])
            BatchPhysicalTableSourceScan(table=[[test-catalog, default, table1]], fields=[id, val, val1, dimid])
          BatchPhysicalExchange(distribution=[broadcast])
            BatchPhysicalCalc(select=[SEARCH(id, Sarg[]) AS $f1])
              BatchPhysicalTableSourceScan(table=[[test-catalog, default, table3]], fields=[id]) 

       

      But it'll throw exception when generate code for it.

      The exception is 

       

       

      java.util.NoSuchElementException
          at com.google.common.collect.ImmutableRangeSet.span(ImmutableRangeSet.java:203)
          at org.apache.calcite.util.Sarg.isComplementedPoints(Sarg.java:148)
          at org.apache.flink.table.planner.codegen.calls.SearchOperatorGen$.generateSearch(SearchOperatorGen.scala:87)
          at org.apache.flink.table.planner.codegen.ExprCodeGenerator.visitCall(ExprCodeGenerator.scala:474)
          at org.apache.flink.table.planner.codegen.ExprCodeGenerator.visitCall(ExprCodeGenerator.scala:57)
          at org.apache.calcite.rex.RexCall.accept(RexCall.java:174)
          at org.apache.flink.table.planner.codegen.ExprCodeGenerator.generateExpression(ExprCodeGenerator.scala:143)
          at org.apache.flink.table.planner.codegen.CalcCodeGenerator$.$anonfun$generateProcessCode$4(CalcCodeGenerator.scala:140)
          at scala.collection.TraversableLike.$anonfun$map$1(TraversableLike.scala:233)
          at scala.collection.mutable.ResizableArray.foreach(ResizableArray.scala:58)
          at scala.collection.mutable.ResizableArray.foreach$(ResizableArray.scala:51)
          at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:47)
          at scala.collection.TraversableLike.map(TraversableLike.scala:233)
          at scala.collection.TraversableLike.map$(TraversableLike.scala:226)
          at scala.collection.AbstractTraversable.map(Traversable.scala:104)
          at org.apache.flink.table.planner.codegen.CalcCodeGenerator$.produceProjectionCode$1(CalcCodeGenerator.scala:140)
          at org.apache.flink.table.planner.codegen.CalcCodeGenerator$.generateProcessCode(CalcCodeGenerator.scala:164)
          at org.apache.flink.table.planner.codegen.CalcCodeGenerator$.generateCalcOperator(CalcCodeGenerator.scala:49)
          at org.apache.flink.table.planner.codegen.CalcCodeGenerator.generateCalcOperator(CalcCodeGenerator.scala)
          at org.apache.flink.table.planner.plan.nodes.exec.common.CommonExecCalc.translateToPlanInternal(CommonExecCalc.java:100)
          at org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase.translateToPlan(ExecNodeBase.java:158)
          at org.apache.flink.table.planner.plan.nodes.exec.ExecEdge.translateToPlan(ExecEdge.java:257)
          at org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecExchange.translateToPlanInternal(BatchExecExchange.java:136) 

      Attachments

        Issue Links

          Activity

            People

              Unassigned Unassigned
              luoyuxia luoyuxia
              Votes:
              0 Vote for this issue
              Watchers:
              2 Start watching this issue

              Dates

                Created:
                Updated: