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

Add support for standard deviation/variance

    Details

    • Type: New Feature
    • Status: Resolved
    • Priority: Major
    • Resolution: Fixed
    • Affects Version/s: None
    • Fix Version/s: 1.3.0
    • Component/s: Table API & SQL
    • Labels:
      None

      Description

      Calcite's AggregateReduceFunctionsRule can convert SQL AVG, STDDEV_POP, STDDEV_SAMP, VAR_POP, VAR_SAMP to sum/count functions. We should add, test and document this rule.

      If we also want to add this aggregates to Table API is up for discussion.

      1. 1.jpg
        133 kB
        Anton Mushin

        Issue Links

          Activity

          Hide
          anmu Anton Mushin added a comment -

          Hello.
          Could I start to resolve this ticket?

          Show
          anmu Anton Mushin added a comment - Hello. Could I start to resolve this ticket?
          Hide
          twalthr Timo Walther added a comment -

          Anton Mushin yes sure. I assigned the issue to you. If you need any help, just ask. Thanks!

          Show
          twalthr Timo Walther added a comment - Anton Mushin yes sure. I assigned the issue to you. If you need any help, just ask. Thanks!
          Hide
          anmu Anton Mushin added a comment -

          Timo Walther,thanks!
          I have any question about this issue.
          1. Am I correct understand in general the test should look like as

          public void testNewAggregationFunctions() throws Exception {
          		//set env ....
          		String sqlQuery = "SELECT AVG(x),STDDEV_POP(x),STDDEV_SAMP(x),VAR_POP(x),VAR_SAMP(x) FROM table";
          		Table result = tableEnv.sql(sqlQuery);
          		/*
          		AVG(x) = SUM(x) / COUNT(x)
          		STDDEV_POP(x) = SQRT( (SUM(x * x) - SUM(x) * SUM(x) / COUNT(x)) / COUNT(x))
          		STDDEV_SAMP(x)= SQRT((SUM(x * x) - SUM(x) * SUM(x) / COUNT(x)) / CASE COUNT(x) WHEN 1 THEN NULL ELSE COUNT(x) - 1 END)
          		VAR_POP(x)= (SUM(x * x) - SUM(x) * SUM(x) / COUNT(x))/ COUNT(x)
          		VAR_SAMP(x) = (SUM(x * x) - SUM(x) * SUM(x) / COUNT(x))/ CASE COUNT(x) WHEN 1 THEN NULL ELSE COUNT(x) - 1 END
          		 */
          		String sqlQuery1 = "SELECT SUM(x)/COUNT(x), " +
          			"SQRT( (SUM(x * x) - SUM(x) * SUM(x) / COUNT(x)) / COUNT(x)), " +
          			"SQRT( (SUM(x * x) - SUM(x) * SUM(x) / COUNT(x)) / CASE COUNT(x) WHEN 1 THEN NULL ELSE COUNT(x) - 1 END), " +
          			"(SUM(x * x) - SUM(x) * SUM(x) / COUNT(x)) / COUNT(x), " +
          			"(SUM(x * x) - SUM(x) * SUM(x) / COUNT(x)) / CASE COUNT(x) WHEN 1 THEN NULL ELSE COUNT(x) - 1 END " +
          			"FROM table";
          		Table result1 = tableEnv.sql(sqlQuery1);
          
          		DataSet<Row> resultSet = tableEnv.toDataSet(result, Row.class);
          		List<Row> results = resultSet.collect();
          		DataSet<Row> expectedResultSet = tableEnv.toDataSet(result1, Row.class);
          		List<Row> expectedResults = expectedResultSet.collect();
                          compareResult(results, expected);
          }
          

          +some single test for each new function?

          2. For support AggregateReduceFunctionsRule I should add to org.apache.flink.api.table.plan.rules.FlinkRuleSets AggregateReduceFunctionsRule.INSTANCE and implement logic STDDEV_POP,STDDEV_SAMP,VAR_POP,VAR_SAMP in package org.apache.flink.api.table.runtime.aggregate like for AvgAggregate, isn't it?
          And define new aggregate functions in other places where it is necessary, for example: org.apache.flink.api.table.validate.FunctionCatalog

          Show
          anmu Anton Mushin added a comment - Timo Walther ,thanks! I have any question about this issue. 1. Am I correct understand in general the test should look like as public void testNewAggregationFunctions() throws Exception { //set env .... String sqlQuery = "SELECT AVG(x),STDDEV_POP(x),STDDEV_SAMP(x),VAR_POP(x),VAR_SAMP(x) FROM table" ; Table result = tableEnv.sql(sqlQuery); /* AVG(x) = SUM(x) / COUNT(x) STDDEV_POP(x) = SQRT( (SUM(x * x) - SUM(x) * SUM(x) / COUNT(x)) / COUNT(x)) STDDEV_SAMP(x)= SQRT((SUM(x * x) - SUM(x) * SUM(x) / COUNT(x)) / CASE COUNT(x) WHEN 1 THEN NULL ELSE COUNT(x) - 1 END) VAR_POP(x)= (SUM(x * x) - SUM(x) * SUM(x) / COUNT(x))/ COUNT(x) VAR_SAMP(x) = (SUM(x * x) - SUM(x) * SUM(x) / COUNT(x))/ CASE COUNT(x) WHEN 1 THEN NULL ELSE COUNT(x) - 1 END */ String sqlQuery1 = "SELECT SUM(x)/COUNT(x), " + "SQRT( (SUM(x * x) - SUM(x) * SUM(x) / COUNT(x)) / COUNT(x)), " + "SQRT( (SUM(x * x) - SUM(x) * SUM(x) / COUNT(x)) / CASE COUNT(x) WHEN 1 THEN NULL ELSE COUNT(x) - 1 END), " + "(SUM(x * x) - SUM(x) * SUM(x) / COUNT(x)) / COUNT(x), " + "(SUM(x * x) - SUM(x) * SUM(x) / COUNT(x)) / CASE COUNT(x) WHEN 1 THEN NULL ELSE COUNT(x) - 1 END " + "FROM table" ; Table result1 = tableEnv.sql(sqlQuery1); DataSet<Row> resultSet = tableEnv.toDataSet(result, Row.class); List<Row> results = resultSet.collect(); DataSet<Row> expectedResultSet = tableEnv.toDataSet(result1, Row.class); List<Row> expectedResults = expectedResultSet.collect(); compareResult(results, expected); } +some single test for each new function? 2. For support AggregateReduceFunctionsRule I should add to org.apache.flink.api.table.plan.rules.FlinkRuleSets AggregateReduceFunctionsRule.INSTANCE and implement logic STDDEV_POP,STDDEV_SAMP,VAR_POP,VAR_SAMP in package org.apache.flink.api.table.runtime.aggregate like for AvgAggregate, isn't it? And define new aggregate functions in other places where it is necessary, for example: org.apache.flink.api.table.validate.FunctionCatalog
          Hide
          twalthr Timo Walther added a comment -

          Hi Anton Mushin,
          1. Yes, the test should look like the one in your example. Since this issue does not implement new runtime functions, it ok to just test some cases to check proper logical translation. We will soon provide a test base so that we can just test the logical translation and don't have to run an entire Flink program. IMHO one large test for all function is also ok. You also don't need to test every data type.

          2. Yes this rule has to be added to the DataSet rules. But we don't need new runtime functions. Calcite translates all aggregates to SUM/COUNT expressions which are already supported in org.apache.flink.api.table.runtime.aggregate. In order to have those functions also in the Table API you need to add them to expressionDsl.scala, ExpressionParser/FunctionCatalog and create expression case classes in aggregations.scala.

          Show
          twalthr Timo Walther added a comment - Hi Anton Mushin , 1. Yes, the test should look like the one in your example. Since this issue does not implement new runtime functions, it ok to just test some cases to check proper logical translation. We will soon provide a test base so that we can just test the logical translation and don't have to run an entire Flink program. IMHO one large test for all function is also ok. You also don't need to test every data type. 2. Yes this rule has to be added to the DataSet rules. But we don't need new runtime functions. Calcite translates all aggregates to SUM/COUNT expressions which are already supported in org.apache.flink.api.table.runtime.aggregate . In order to have those functions also in the Table API you need to add them to expressionDsl.scala , ExpressionParser / FunctionCatalog and create expression case classes in aggregations.scala .
          Hide
          anmu Anton Mushin added a comment -

          I have some problem.
          I added rule in org.apache.flink.api.table.plan.rules.FlinkRuleSets#DATASET_OPT_RULES

          .....
                 // aggregate union rule
              AggregateUnionAggregateRule.INSTANCE,
          
              AggregateReduceFunctionsRule.INSTANCE,
          ......
          

          I create case classes in org.apache.flink.api.table.expressions.aggregations.scala look like as

          case class StddevPop(child: Expression) extends Aggregation {
            override def toString = s"stddev_pop($child)"
          
            override private[flink] def toAggCall(name: String)(implicit relBuilder: RelBuilder): AggCall = {
              relBuilder.aggregateCall(SqlStdOperatorTable.STDDEV_POP, false, null, name, child.toRexNode)
            }
          
            override private[flink] def resultType = child.resultType
          
            override private[flink] def validateInput =
              TypeCheckUtils.assertNumericExpr(child.resultType, "stddev_pop")
          }
          

          and described new functions in org.apache.flink.api.table.validate.FunctionCatalog#builtInFunctions

          ......
          "sum" -> classOf[Sum],
              "stddev_pop" -> classOf[StddevPop],
              "stddev_samp" -> classOf[StddevSamp],
              "var_pop" -> classOf[VarPop],
              "var_samp" -> classOf[VarSamp],
          .......
          

          and in org.apache.flink.api.table.validate.BasicOperatorTable#builtInSqlOperators

          .........
          // AGGREGATE OPERATORS
          ........
              SqlStdOperatorTable.AVG,
              SqlStdOperatorTable.STDDEV_POP,
              SqlStdOperatorTable.STDDEV_SAMP,
              SqlStdOperatorTable.VAR_POP,
              SqlStdOperatorTable.VAR_SAMP,
          ........
          

          Also I described functions in org.apache.flink.api.table.expressions.ExpressionParser lool like as

          lazy val STDDEV_POP: Keyword = Keyword("stddev_pop")
          .....
          lazy val suffixStddevPop: PackratParser[Expression] =
              composite <~ "." ~ STDDEV_POP ~ opt("()") ^^ { e => StddevPop(e) }
          
          ......
          lazy val prefixStddevPop: PackratParser[Expression] =
              STDDEV_POP ~ "(" ~> expression <~ ")" ^^ { e => StddevPop(e) }
          
          // and it added in ExpressionParser#functionIdent, ExpressionParser#suffixed, ExpressionParser#prefixed as for other aggregate functions 
          

          and I added them into org.apache.flink.api.scala.table.expressionDsl.scala

          ......
          def stddev_pop = StddevPop(expr)
            def stddev_samp = StddevSamp(expr)
            def var_pop = VarPop(expr)
            def var_samp = VarSamp(expr)
          

          Then I'm execute next sql query: SELECT STDDEV_POP(_1),STDDEV_SAMP(_1),VAR_SAMP(_1),VAR_POP(_1) FROM table
          In org.apache.flink.api.table.BatchTableEnvironment#translate I getting dataset plan as

          LogicalAggregate(group=[{}], EXPR$0=[STDDEV_POP($0)], EXPR$1=[STDDEV_SAMP($0)], EXPR$2=[VAR_SAMP($0)], EXPR$3=[VAR_POP($0)])
            LogicalProject(_1=[$0])
              LogicalTableScan(table=[[_DataSetTable_0]])
          

          When org.apache.flink.api.table.plan.nodes.dataset.DataSetAggregate#translateToPlan occur then org.apache.flink.api.table.runtime.aggregate.AvgAggregate is called for each new function and becouse of it avg calculate for each function
          Could you suggest what I might miss?

          Show
          anmu Anton Mushin added a comment - I have some problem. I added rule in org.apache.flink.api.table.plan.rules.FlinkRuleSets#DATASET_OPT_RULES ..... // aggregate union rule AggregateUnionAggregateRule.INSTANCE, AggregateReduceFunctionsRule.INSTANCE, ...... I create case classes in org.apache.flink.api.table.expressions.aggregations.scala look like as case class StddevPop(child: Expression) extends Aggregation { override def toString = s "stddev_pop($child)" override private [flink] def toAggCall(name: String )(implicit relBuilder: RelBuilder): AggCall = { relBuilder.aggregateCall(SqlStdOperatorTable.STDDEV_POP, false , null , name, child.toRexNode) } override private [flink] def resultType = child.resultType override private [flink] def validateInput = TypeCheckUtils.assertNumericExpr(child.resultType, "stddev_pop" ) } and described new functions in org.apache.flink.api.table.validate.FunctionCatalog#builtInFunctions ...... "sum" -> classOf[Sum], "stddev_pop" -> classOf[StddevPop], "stddev_samp" -> classOf[StddevSamp], "var_pop" -> classOf[VarPop], "var_samp" -> classOf[VarSamp], ....... and in org.apache.flink.api.table.validate.BasicOperatorTable#builtInSqlOperators ......... // AGGREGATE OPERATORS ........ SqlStdOperatorTable.AVG, SqlStdOperatorTable.STDDEV_POP, SqlStdOperatorTable.STDDEV_SAMP, SqlStdOperatorTable.VAR_POP, SqlStdOperatorTable.VAR_SAMP, ........ Also I described functions in org.apache.flink.api.table.expressions.ExpressionParser lool like as lazy val STDDEV_POP: Keyword = Keyword( "stddev_pop" ) ..... lazy val suffixStddevPop: PackratParser[Expression] = composite <~ "." ~ STDDEV_POP ~ opt( "()" ) ^^ { e => StddevPop(e) } ...... lazy val prefixStddevPop: PackratParser[Expression] = STDDEV_POP ~ "(" ~> expression <~ ")" ^^ { e => StddevPop(e) } // and it added in ExpressionParser#functionIdent, ExpressionParser#suffixed, ExpressionParser#prefixed as for other aggregate functions and I added them into org.apache.flink.api.scala.table.expressionDsl.scala ...... def stddev_pop = StddevPop(expr) def stddev_samp = StddevSamp(expr) def var_pop = VarPop(expr) def var_samp = VarSamp(expr) Then I'm execute next sql query: SELECT STDDEV_POP(_1),STDDEV_SAMP(_1),VAR_SAMP(_1),VAR_POP(_1) FROM table In org.apache.flink.api.table.BatchTableEnvironment#translate I getting dataset plan as LogicalAggregate(group=[{}], EXPR$0=[STDDEV_POP($0)], EXPR$1=[STDDEV_SAMP($0)], EXPR$2=[VAR_SAMP($0)], EXPR$3=[VAR_POP($0)]) LogicalProject(_1=[$0]) LogicalTableScan(table=[[_DataSetTable_0]]) When org.apache.flink.api.table.plan.nodes.dataset.DataSetAggregate#translateToPlan occur then org.apache.flink.api.table.runtime.aggregate.AvgAggregate is called for each new function and becouse of it avg calculate for each function Could you suggest what I might miss?
          Hide
          twalthr Timo Walther added a comment -

          Your implementation looks good so far. The output of BatchTableEnvironment#translate should actually be a tree of DataSetAggregate, DataSetCalc nodes. Are you using explain to get the plan above? It might only print the logical plan. You can have a look at ExpressionReductionTest how you can get the optimized plan. This will be easier when https://github.com/apache/flink/pull/2595 is merged.

          Show
          twalthr Timo Walther added a comment - Your implementation looks good so far. The output of BatchTableEnvironment#translate should actually be a tree of DataSetAggregate , DataSetCalc nodes. Are you using explain to get the plan above? It might only print the logical plan. You can have a look at ExpressionReductionTest how you can get the optimized plan. This will be easier when https://github.com/apache/flink/pull/2595 is merged.
          Hide
          anmu Anton Mushin added a comment - - edited

          I used RelOptUtil.toString(relNode) for getting plan

          org.apache.flink.api.table.BatchTableEnvironment#translate
           ..........
          val dataSetPlan = try {
                optProgram.run(getPlanner, decorPlan, flinkOutputProps)
              }
              catch {
                ..........
              }
              print(s"\n${RelOptUtil.toString(dataSetPlan)}\n${RelOptUtil.toString(relNode)}")
              dataSetPlan match {
                case node: DataSetRel =>
                  node.translateToPlan(
                    this,
                    Some(tpe.asInstanceOf[TypeInformation[Any]])
                  ).asInstanceOf[DataSet[A]]
                case _ => ???
              }
            }
          

          I getting in the output

          DataSetAggregate(select=[STDDEV_POP(_1) AS EXPR$0, STDDEV_SAMP(_1) AS EXPR$1, VAR_SAMP(_1) AS EXPR$2, VAR_POP(_1) AS EXPR$3])
            DataSetScan(table=[[_DataSetTable_0]])
          
          LogicalAggregate(group=[{}], EXPR$0=[STDDEV_POP($0)], EXPR$1=[STDDEV_SAMP($0)], EXPR$2=[VAR_SAMP($0)], EXPR$3=[VAR_POP($0)])
            LogicalProject(_1=[$0])
              LogicalTableScan(table=[[_DataSetTable_0]])
          
          Show
          anmu Anton Mushin added a comment - - edited I used RelOptUtil.toString(relNode) for getting plan org.apache.flink.api.table.BatchTableEnvironment#translate .......... val dataSetPlan = try { optProgram.run(getPlanner, decorPlan, flinkOutputProps) } catch { .......... } print(s "\n${RelOptUtil.toString(dataSetPlan)}\n${RelOptUtil.toString(relNode)}" ) dataSetPlan match { case node: DataSetRel => node.translateToPlan( this , Some(tpe.asInstanceOf[TypeInformation[Any]]) ).asInstanceOf[DataSet[A]] case _ => ??? } } I getting in the output DataSetAggregate(select=[STDDEV_POP(_1) AS EXPR$0, STDDEV_SAMP(_1) AS EXPR$1, VAR_SAMP(_1) AS EXPR$2, VAR_POP(_1) AS EXPR$3]) DataSetScan(table=[[_DataSetTable_0]]) LogicalAggregate(group=[{}], EXPR$0=[STDDEV_POP($0)], EXPR$1=[STDDEV_SAMP($0)], EXPR$2=[VAR_SAMP($0)], EXPR$3=[VAR_POP($0)]) LogicalProject(_1=[$0]) LogicalTableScan(table=[[_DataSetTable_0]])
          Hide
          twalthr Timo Walther added a comment - - edited

          Ok, it seems that the AggregateReduceFunctionsRule doesn't work correctly. It would be great if you could figure out why. Can you check if AggregateReduceFunctionsRule.matches is called? Maybe we have to adapt the RelOptRuleOperand condition or Calcite has a problem that needs to be reported on the Calcite mailing list.

          But didn't you say that AvgAggregate is called so the translation happened even though it might not be in the part of the expression string?

          Show
          twalthr Timo Walther added a comment - - edited Ok, it seems that the AggregateReduceFunctionsRule doesn't work correctly. It would be great if you could figure out why. Can you check if AggregateReduceFunctionsRule.matches is called? Maybe we have to adapt the RelOptRuleOperand condition or Calcite has a problem that needs to be reported on the Calcite mailing list. But didn't you say that AvgAggregate is called so the translation happened even though it might not be in the part of the expression string?
          Hide
          anmu Anton Mushin added a comment - - edited

          Can you check if AggregateReduceFunctionsRule.matches is called?

          yes it was called. It was called from org.apache.calcite.plan.volcano.VolcanoRuleCall#matchRecurse when executing org.apache.calcite.plan.volcano.VolcanoPlanner#findBestExp on OPTIMIZE phase. (see screenshot for check)
          But how best rule selected DatasetSetAggregate object: Aggregate(}select:(STDDEV_POP(_1) AS EXPR$0, STDDEV_SAMP(_1) AS EXPR$1, VAR_SAMP(_1) AS EXPR$2, VAR_POP(_1) AS EXPR$3)) and this plan return as dataSetPlan into org.apache.flink.api.table.BatchTableEnvironment#translate

          Show
          anmu Anton Mushin added a comment - - edited Can you check if AggregateReduceFunctionsRule.matches is called? yes it was called. It was called from org.apache.calcite.plan.volcano.VolcanoRuleCall#matchRecurse when executing org.apache.calcite.plan.volcano.VolcanoPlanner#findBestExp on OPTIMIZE phase. ( see screenshot for check ) But how best rule selected DatasetSetAggregate object: Aggregate(}select:(STDDEV_POP(_1) AS EXPR$0, STDDEV_SAMP(_1) AS EXPR$1, VAR_SAMP(_1) AS EXPR$2, VAR_POP(_1) AS EXPR$3)) and this plan return as dataSetPlan into org.apache.flink.api.table.BatchTableEnvironment#translate
          Hide
          twalthr Timo Walther added a comment -

          Maybe the cost function of DataSetAggregate is faulty. Could you post a link to your branch? I will have a look at it.

          Show
          twalthr Timo Walther added a comment - Maybe the cost function of DataSetAggregate is faulty. Could you post a link to your branch? I will have a look at it.
          Show
          anmu Anton Mushin added a comment - Yes, of course. link to commit: https://github.com/ex00/flink/commit/8f802db568ce271bfb1597f7c2a29cc0d00f55f6
          Hide
          anmu Anton Mushin added a comment -

          I'm think will probably worth consider types of aggregate functions in gorg.apache.flink.api.table.plan.nodes.dataset.DataSetAggregate#computeSelfCost
          What do you think about it?

          Show
          anmu Anton Mushin added a comment - I'm think will probably worth consider types of aggregate functions in gorg.apache.flink.api.table.plan.nodes.dataset.DataSetAggregate#computeSelfCost What do you think about it?
          Hide
          twalthr Timo Walther added a comment -

          Yes, you are right. We have to make the costs for STDDEV_POP, etc. very high. So if one of those aggregation functions is contained, than planner.getCostFactory.makeInfiniteCost().

          Show
          twalthr Timo Walther added a comment - Yes, you are right. We have to make the costs for STDDEV_POP, etc. very high. So if one of those aggregation functions is contained, than planner.getCostFactory.makeInfiniteCost() .
          Hide
          anmu Anton Mushin added a comment -

          I changed method org.apache.flink.api.table.plan.nodes.dataset.DataSetAggregate#computeSelfCost
          it is impermanent implementation for examle

          override def computeSelfCost (planner: RelOptPlanner, metadata: RelMetadataQuery): RelOptCost = {
              val child = this.getInput
              val rowCnt = metadata.getRowCount(child)
              val rowSize = this.estimateRowSize(child.getRowType)
              val aggCnt = this.namedAggregates.size
              var resultCost = planner.getCostFactory.makeCost(rowCnt, rowCnt * aggCnt, rowCnt * rowSize)
              this.namedAggregates.foreach(x=>{
                x.getKey.getAggregation.getKind match {
                  case SqlKind.STDDEV_POP =>
                    resultCost = resultCost.plus(planner.getCostFactory.makeCost(rowCnt, rowCnt * aggCnt, rowCnt * rowSize))
                  case SqlKind.STDDEV_SAMP =>
                    resultCost = resultCost.plus(planner.getCostFactory.makeCost(rowCnt, rowCnt * aggCnt, rowCnt * rowSize))
                  case SqlKind.VAR_SAMP =>
                    resultCost = resultCost.plus(planner.getCostFactory.makeCost(rowCnt, rowCnt * aggCnt, rowCnt * rowSize))
                  case SqlKind.VAR_POP =>
                    resultCost = resultCost.plus(planner.getCostFactory.makeCost(rowCnt, rowCnt * aggCnt, rowCnt * rowSize))
                  case default => None
                }
              })
              resultCost
            }
          

          and i got next plan:

          DataSetCalc(select=[CAST(POWER(/(-($f0, /(*($f1, $f1), $f2)), $f2), 0.5)) AS $f0, CAST(POWER(/(-($f3, /(*($f1, $f1), $f2)), CASE(=($f2, 1), null, -($f2, 1))), 0.5)) AS $f1, CAST(/(-($f4, /(*($f1, $f1), $f2)), CASE(=($f2, 1), null, -($f2, 1)))) AS $f2, CAST(/(-($f5, /(*($f1, $f1), $f2)), $f2)) AS $f3])
            DataSetAggregate(select=[SUM($f6) AS $f0, SUM(_1) AS $f1, COUNT(_1) AS $f2, SUM($f7) AS $f3, SUM($f8) AS $f4, SUM($f9) AS $f5])
              DataSetCalc(select=[_1, _2, _3, _4, _5, _6])
                DataSetScan(table=[[_DataSetTable_0]])
          
          LogicalAggregate(group=[{}], EXPR$0=[STDDEV_POP($0)], EXPR$1=[STDDEV_SAMP($0)], EXPR$2=[VAR_SAMP($0)], EXPR$3=[VAR_POP($0)])
            LogicalProject(_1=[$0])
              LogicalTableScan(table=[[_DataSetTable_0]])
          
          Show
          anmu Anton Mushin added a comment - I changed method org.apache.flink.api.table.plan.nodes.dataset.DataSetAggregate#computeSelfCost it is impermanent implementation for examle override def computeSelfCost (planner: RelOptPlanner, metadata: RelMetadataQuery): RelOptCost = { val child = this .getInput val rowCnt = metadata.getRowCount(child) val rowSize = this .estimateRowSize(child.getRowType) val aggCnt = this .namedAggregates.size var resultCost = planner.getCostFactory.makeCost(rowCnt, rowCnt * aggCnt, rowCnt * rowSize) this .namedAggregates.foreach(x=>{ x.getKey.getAggregation.getKind match { case SqlKind.STDDEV_POP => resultCost = resultCost.plus(planner.getCostFactory.makeCost(rowCnt, rowCnt * aggCnt, rowCnt * rowSize)) case SqlKind.STDDEV_SAMP => resultCost = resultCost.plus(planner.getCostFactory.makeCost(rowCnt, rowCnt * aggCnt, rowCnt * rowSize)) case SqlKind.VAR_SAMP => resultCost = resultCost.plus(planner.getCostFactory.makeCost(rowCnt, rowCnt * aggCnt, rowCnt * rowSize)) case SqlKind.VAR_POP => resultCost = resultCost.plus(planner.getCostFactory.makeCost(rowCnt, rowCnt * aggCnt, rowCnt * rowSize)) case default => None } }) resultCost } and i got next plan: DataSetCalc(select=[CAST(POWER(/(-($f0, /(*($f1, $f1), $f2)), $f2), 0.5)) AS $f0, CAST(POWER(/(-($f3, /(*($f1, $f1), $f2)), CASE(=($f2, 1), null, -($f2, 1))), 0.5)) AS $f1, CAST(/(-($f4, /(*($f1, $f1), $f2)), CASE(=($f2, 1), null, -($f2, 1)))) AS $f2, CAST(/(-($f5, /(*($f1, $f1), $f2)), $f2)) AS $f3]) DataSetAggregate(select=[SUM($f6) AS $f0, SUM(_1) AS $f1, COUNT(_1) AS $f2, SUM($f7) AS $f3, SUM($f8) AS $f4, SUM($f9) AS $f5]) DataSetCalc(select=[_1, _2, _3, _4, _5, _6]) DataSetScan(table=[[_DataSetTable_0]]) LogicalAggregate(group=[{}], EXPR$0=[STDDEV_POP($0)], EXPR$1=[STDDEV_SAMP($0)], EXPR$2=[VAR_SAMP($0)], EXPR$3=[VAR_POP($0)]) LogicalProject(_1=[$0]) LogicalTableScan(table=[[_DataSetTable_0]])
          Hide
          twalthr Timo Walther added a comment -

          I think a even nicer solution would be to check the aggregation function type already in `DataSetAggregateRule`, so that the rule never matches in those cases.

          Show
          twalthr Timo Walther added a comment - I think a even nicer solution would be to check the aggregation function type already in `DataSetAggregateRule`, so that the rule never matches in those cases.
          Hide
          anmu Anton Mushin added a comment -

          Do you mean check in org.apache.flink.api.table.plan.rules.dataSet.DataSetAggregateRule#matches? or convert to other RelNode in org.apache.flink.api.table.plan.rules.dataSet.DataSetAggregateRule#convert

          Show
          anmu Anton Mushin added a comment - Do you mean check in org.apache.flink.api.table.plan.rules.dataSet.DataSetAggregateRule#matches ? or convert to other RelNode in org.apache.flink.api.table.plan.rules.dataSet.DataSetAggregateRule#convert
          Hide
          twalthr Timo Walther added a comment -

          Yes, I mean matches. DataSetAggregate does not support those aggregations so the rule should never match.

          Show
          twalthr Timo Walther added a comment - Yes, I mean matches . DataSetAggregate does not support those aggregations so the rule should never match.
          Hide
          anmu Anton Mushin added a comment -

          I tried check function in org.apache.flink.api.table.plan.rules.dataSet.DataSetAggregateRule#matches, but something went wrong
          I did so

          org.apache.flink.api.table.plan.rules.dataSet.DataSetAggregateRule
          override def matches(call: RelOptRuleCall): Boolean = {
              val agg: LogicalAggregate = call.rel(0).asInstanceOf[LogicalAggregate]
          
              // check if we have distinct aggregates
              val distinctAggs = agg.getAggCallList.exists(_.isDistinct)
              if (distinctAggs) {
                throw new TableException("DISTINCT aggregates are currently not supported.")
              }
          
              // check if we have grouping sets
              val groupSets = agg.getGroupSets.size() != 1 || agg.getGroupSets.get(0) != agg.getGroupSet
              if (groupSets || agg.indicator) {
                throw new TableException("GROUPING SETS are currently not supported.")
              }
          
              (!distinctAggs && !groupSets && !agg.indicator) && !AggregateReduceFunctionsRule.INSTANCE.matches(call)
            }
          

          And I got next plan and exception:

          DataSetCalc(select=[CAST(/(-(CASE(=($f1, 0), null, $f0), /(*(CASE(=($f3, 0), null, $f2), CASE(=($f3, 0), null, $f2)), $f3)), CASE(=($f3, 1), null, -($f3, 1)))) AS $f0, CAST(/(-(CASE(=($f5, 0), null, $f4), /(*(CASE(=($f7, 0), null, $f6), CASE(=($f7, 0), null, $f6)), $f7)), CASE(=($f7, 1), null, -($f7, 1)))) AS $f1, CAST(/(-(CASE(=($f9, 0), null, $f8), /(*(CASE(=($f11, 0), null, $f10), CASE(=($f11, 0), null, $f10)), $f11)), CASE(=($f11, 1), null, -($f11, 1)))) AS $f2, CAST(/(-(CASE(=($f13, 0), null, $f12), /(*(CASE(=($f15, 0), null, $f14), CASE(=($f15, 0), null, $f14)), $f15)), CASE(=($f15, 1), null, -($f15, 1)))) AS $f3, CAST(/(-(CASE(=($f17, 0), null, $f16), /(*(CASE(=($f19, 0), null, $f18), CASE(=($f19, 0), null, $f18)), $f19)), CASE(=($f19, 1), null, -($f19, 1)))) AS $f4, CAST(/(-(CASE(=($f21, 0), null, $f20), /(*(CASE(=($f23, 0), null, $f22), CASE(=($f23, 0), null, $f22)), $f23)), CASE(=($f23, 1), null, -($f23, 1)))) AS $f5])
            DataSetAggregate(select=[$SUM0($f6) AS $f0, COUNT($f6) AS $f1, $SUM0(_1) AS $f2, COUNT(_1) AS $f3, $SUM0($f7) AS $f4, COUNT($f7) AS $f5, $SUM0(_2) AS $f6, COUNT(_2) AS $f7, $SUM0($f8) AS $f8, COUNT($f8) AS $f9, $SUM0(_3) AS $f10, COUNT(_3) AS $f11, $SUM0($f9) AS $f12, COUNT($f9) AS $f13, $SUM0(_4) AS $f14, COUNT(_4) AS $f15, $SUM0($f10) AS $f16, COUNT($f10) AS $f17, $SUM0(_5) AS $f18, COUNT(_5) AS $f19, $SUM0($f11) AS $f20, COUNT($f11) AS $f21, $SUM0(_6) AS $f22, COUNT(_6) AS $f23])
              DataSetCalc(select=[_1, _2, _3, _4, _5, _6])
                DataSetScan(table=[[_DataSetTable_0]])
          
          org.apache.flink.api.table.TableException: Type NULL is not supported. Null values must have a supported type.
          
          	at org.apache.flink.api.table.FlinkTypeFactory$.toTypeInfo(FlinkTypeFactory.scala:128)
          	at org.apache.flink.api.table.codegen.CodeGenerator.visitLiteral(CodeGenerator.scala:553)
          	at org.apache.flink.api.table.codegen.CodeGenerator.visitLiteral(CodeGenerator.scala:56)
          	at org.apache.calcite.rex.RexLiteral.accept(RexLiteral.java:658)
          	at org.apache.flink.api.table.codegen.CodeGenerator$$anonfun$14.apply(CodeGenerator.scala:675)
          	at org.apache.flink.api.table.codegen.CodeGenerator$$anonfun$14.apply(CodeGenerator.scala:675)
          	at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244)
          	at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244)
          	at scala.collection.Iterator$class.foreach(Iterator.scala:727)
          	at scala.collection.AbstractIterator.foreach(Iterator.scala:1157)
          	at scala.collection.IterableLike$class.foreach(IterableLike.scala:72)
          	at scala.collection.AbstractIterable.foreach(Iterable.scala:54)
          	at scala.collection.TraversableLike$class.map(TraversableLike.scala:244)
          	at scala.collection.AbstractTraversable.map(Traversable.scala:105)
          	at org.apache.flink.api.table.codegen.CodeGenerator.visitCall(CodeGenerator.scala:675)
          	at org.apache.flink.api.table.codegen.CodeGenerator.visitCall(CodeGenerator.scala:56)
          	at org.apache.calcite.rex.RexCall.accept(RexCall.java:108)
          	at org.apache.flink.api.table.codegen.CodeGenerator$$anonfun$14.apply(CodeGenerator.scala:675)
          	at org.apache.flink.api.table.codegen.CodeGenerator$$anonfun$14.apply(CodeGenerator.scala:675)
          	at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244)
          	at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244)
          	at scala.collection.Iterator$class.foreach(Iterator.scala:727)
          	at scala.collection.AbstractIterator.foreach(Iterator.scala:1157)
          	at scala.collection.IterableLike$class.foreach(IterableLike.scala:72)
          	at scala.collection.AbstractIterable.foreach(Iterable.scala:54)
          	at scala.collection.TraversableLike$class.map(TraversableLike.scala:244)
          	at scala.collection.AbstractTraversable.map(Traversable.scala:105)
          	at org.apache.flink.api.table.codegen.CodeGenerator.visitCall(CodeGenerator.scala:675)
          	at org.apache.flink.api.table.codegen.CodeGenerator.visitCall(CodeGenerator.scala:56)
          	at org.apache.calcite.rex.RexCall.accept(RexCall.java:108)
          	at org.apache.flink.api.table.codegen.CodeGenerator$$anonfun$14.apply(CodeGenerator.scala:675)
          	at org.apache.flink.api.table.codegen.CodeGenerator$$anonfun$14.apply(CodeGenerator.scala:675)
          	at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244)
          	at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244)
          	at scala.collection.Iterator$class.foreach(Iterator.scala:727)
          	at scala.collection.AbstractIterator.foreach(Iterator.scala:1157)
          	at scala.collection.IterableLike$class.foreach(IterableLike.scala:72)
          	at scala.collection.AbstractIterable.foreach(Iterable.scala:54)
          	at scala.collection.TraversableLike$class.map(TraversableLike.scala:244)
          	at scala.collection.AbstractTraversable.map(Traversable.scala:105)
          	at org.apache.flink.api.table.codegen.CodeGenerator.visitCall(CodeGenerator.scala:675)
          	at org.apache.flink.api.table.codegen.CodeGenerator.visitCall(CodeGenerator.scala:56)
          	at org.apache.calcite.rex.RexCall.accept(RexCall.java:108)
          	at org.apache.flink.api.table.codegen.CodeGenerator$$anonfun$14.apply(CodeGenerator.scala:675)
          	at org.apache.flink.api.table.codegen.CodeGenerator$$anonfun$14.apply(CodeGenerator.scala:675)
          	at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244)
          	at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244)
          	at scala.collection.Iterator$class.foreach(Iterator.scala:727)
          	at scala.collection.AbstractIterator.foreach(Iterator.scala:1157)
          	at scala.collection.IterableLike$class.foreach(IterableLike.scala:72)
          	at scala.collection.AbstractIterable.foreach(Iterable.scala:54)
          	at scala.collection.TraversableLike$class.map(TraversableLike.scala:244)
          	at scala.collection.AbstractTraversable.map(Traversable.scala:105)
          	at org.apache.flink.api.table.codegen.CodeGenerator.visitCall(CodeGenerator.scala:675)
          	at org.apache.flink.api.table.codegen.CodeGenerator.visitCall(CodeGenerator.scala:56)
          	at org.apache.calcite.rex.RexCall.accept(RexCall.java:108)
          	at org.apache.flink.api.table.codegen.CodeGenerator.generateExpression(CodeGenerator.scala:181)
          	at org.apache.flink.api.table.codegen.CodeGenerator$$anonfun$5.apply(CodeGenerator.scala:300)
          	at org.apache.flink.api.table.codegen.CodeGenerator$$anonfun$5.apply(CodeGenerator.scala:300)
          	at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244)
          	at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244)
          	at scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)
          	at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:47)
          	at scala.collection.TraversableLike$class.map(TraversableLike.scala:244)
          	at scala.collection.AbstractTraversable.map(Traversable.scala:105)
          	at org.apache.flink.api.table.codegen.CodeGenerator.generateResultExpression(CodeGenerator.scala:300)
          	at org.apache.flink.api.table.plan.nodes.FlinkCalc$class.functionBody(FlinkCalc.scala:52)
          	at org.apache.flink.api.table.plan.nodes.dataset.DataSetCalc.functionBody(DataSetCalc.scala:39)
          	at org.apache.flink.api.table.plan.nodes.dataset.DataSetCalc.translateToPlan(DataSetCalc.scala:108)
          	at org.apache.flink.api.table.BatchTableEnvironment.translate(BatchTableEnvironment.scala:274)
          	at org.apache.flink.api.scala.table.BatchTableEnvironment.toDataSet(BatchTableEnvironment.scala:139)
          	at org.apache.flink.api.scala.table.TableConversions.toDataSet(TableConversions.scala:41)
          	at org.apache.flink.api.scala.batch.sql.AggregationsITCase.testVarSampAggregate(AggregationsITCase.scala:369)
          	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:497)
          	at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:47)
          	at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
          	at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:44)
          	at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17)
          	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:271)
          	at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:70)
          	at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:50)
          	at org.junit.runners.ParentRunner$3.run(ParentRunner.java:238)
          	at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:63)
          	at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:236)
          	at org.junit.runners.ParentRunner.access$000(ParentRunner.java:53)
          	at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:229)
          	at org.junit.runners.ParentRunner.run(ParentRunner.java:309)
          	at org.junit.runners.Suite.runChild(Suite.java:127)
          	at org.junit.runners.Suite.runChild(Suite.java:26)
          	at org.junit.runners.ParentRunner$3.run(ParentRunner.java:238)
          	at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:63)
          	at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:236)
          	at org.junit.runners.ParentRunner.access$000(ParentRunner.java:53)
          	at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:229)
          	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.runners.ParentRunner.run(ParentRunner.java:309)
          	at org.junit.runner.JUnitCore.run(JUnitCore.java:160)
          

          then I remove !AggregateReduceFunctionsRule.INSTANCE.matches(call) and return this code, tests is passed.
          I search resolution this problem, do you have any ideas about it?

          Show
          anmu Anton Mushin added a comment - I tried check function in org.apache.flink.api.table.plan.rules.dataSet.DataSetAggregateRule#matches , but something went wrong I did so org.apache.flink.api.table.plan.rules.dataSet.DataSetAggregateRule override def matches(call: RelOptRuleCall): Boolean = { val agg: LogicalAggregate = call.rel(0).asInstanceOf[LogicalAggregate] // check if we have distinct aggregates val distinctAggs = agg.getAggCallList.exists(_.isDistinct) if (distinctAggs) { throw new TableException( "DISTINCT aggregates are currently not supported." ) } // check if we have grouping sets val groupSets = agg.getGroupSets.size() != 1 || agg.getGroupSets.get(0) != agg.getGroupSet if (groupSets || agg.indicator) { throw new TableException( "GROUPING SETS are currently not supported." ) } (!distinctAggs && !groupSets && !agg.indicator) && !AggregateReduceFunctionsRule.INSTANCE.matches(call) } And I got next plan and exception: DataSetCalc(select=[CAST(/(-(CASE(=($f1, 0), null, $f0), /(*(CASE(=($f3, 0), null, $f2), CASE(=($f3, 0), null, $f2)), $f3)), CASE(=($f3, 1), null, -($f3, 1)))) AS $f0, CAST(/(-(CASE(=($f5, 0), null, $f4), /(*(CASE(=($f7, 0), null, $f6), CASE(=($f7, 0), null, $f6)), $f7)), CASE(=($f7, 1), null, -($f7, 1)))) AS $f1, CAST(/(-(CASE(=($f9, 0), null, $f8), /(*(CASE(=($f11, 0), null, $f10), CASE(=($f11, 0), null, $f10)), $f11)), CASE(=($f11, 1), null, -($f11, 1)))) AS $f2, CAST(/(-(CASE(=($f13, 0), null, $f12), /(*(CASE(=($f15, 0), null, $f14), CASE(=($f15, 0), null, $f14)), $f15)), CASE(=($f15, 1), null, -($f15, 1)))) AS $f3, CAST(/(-(CASE(=($f17, 0), null, $f16), /(*(CASE(=($f19, 0), null, $f18), CASE(=($f19, 0), null, $f18)), $f19)), CASE(=($f19, 1), null, -($f19, 1)))) AS $f4, CAST(/(-(CASE(=($f21, 0), null, $f20), /(*(CASE(=($f23, 0), null, $f22), CASE(=($f23, 0), null, $f22)), $f23)), CASE(=($f23, 1), null, -($f23, 1)))) AS $f5]) DataSetAggregate(select=[$SUM0($f6) AS $f0, COUNT($f6) AS $f1, $SUM0(_1) AS $f2, COUNT(_1) AS $f3, $SUM0($f7) AS $f4, COUNT($f7) AS $f5, $SUM0(_2) AS $f6, COUNT(_2) AS $f7, $SUM0($f8) AS $f8, COUNT($f8) AS $f9, $SUM0(_3) AS $f10, COUNT(_3) AS $f11, $SUM0($f9) AS $f12, COUNT($f9) AS $f13, $SUM0(_4) AS $f14, COUNT(_4) AS $f15, $SUM0($f10) AS $f16, COUNT($f10) AS $f17, $SUM0(_5) AS $f18, COUNT(_5) AS $f19, $SUM0($f11) AS $f20, COUNT($f11) AS $f21, $SUM0(_6) AS $f22, COUNT(_6) AS $f23]) DataSetCalc(select=[_1, _2, _3, _4, _5, _6]) DataSetScan(table=[[_DataSetTable_0]]) org.apache.flink.api.table.TableException: Type NULL is not supported. Null values must have a supported type. at org.apache.flink.api.table.FlinkTypeFactory$.toTypeInfo(FlinkTypeFactory.scala:128) at org.apache.flink.api.table.codegen.CodeGenerator.visitLiteral(CodeGenerator.scala:553) at org.apache.flink.api.table.codegen.CodeGenerator.visitLiteral(CodeGenerator.scala:56) at org.apache.calcite.rex.RexLiteral.accept(RexLiteral.java:658) at org.apache.flink.api.table.codegen.CodeGenerator$$anonfun$14.apply(CodeGenerator.scala:675) at org.apache.flink.api.table.codegen.CodeGenerator$$anonfun$14.apply(CodeGenerator.scala:675) at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244) at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244) at scala.collection.Iterator$class.foreach(Iterator.scala:727) at scala.collection.AbstractIterator.foreach(Iterator.scala:1157) at scala.collection.IterableLike$class.foreach(IterableLike.scala:72) at scala.collection.AbstractIterable.foreach(Iterable.scala:54) at scala.collection.TraversableLike$class.map(TraversableLike.scala:244) at scala.collection.AbstractTraversable.map(Traversable.scala:105) at org.apache.flink.api.table.codegen.CodeGenerator.visitCall(CodeGenerator.scala:675) at org.apache.flink.api.table.codegen.CodeGenerator.visitCall(CodeGenerator.scala:56) at org.apache.calcite.rex.RexCall.accept(RexCall.java:108) at org.apache.flink.api.table.codegen.CodeGenerator$$anonfun$14.apply(CodeGenerator.scala:675) at org.apache.flink.api.table.codegen.CodeGenerator$$anonfun$14.apply(CodeGenerator.scala:675) at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244) at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244) at scala.collection.Iterator$class.foreach(Iterator.scala:727) at scala.collection.AbstractIterator.foreach(Iterator.scala:1157) at scala.collection.IterableLike$class.foreach(IterableLike.scala:72) at scala.collection.AbstractIterable.foreach(Iterable.scala:54) at scala.collection.TraversableLike$class.map(TraversableLike.scala:244) at scala.collection.AbstractTraversable.map(Traversable.scala:105) at org.apache.flink.api.table.codegen.CodeGenerator.visitCall(CodeGenerator.scala:675) at org.apache.flink.api.table.codegen.CodeGenerator.visitCall(CodeGenerator.scala:56) at org.apache.calcite.rex.RexCall.accept(RexCall.java:108) at org.apache.flink.api.table.codegen.CodeGenerator$$anonfun$14.apply(CodeGenerator.scala:675) at org.apache.flink.api.table.codegen.CodeGenerator$$anonfun$14.apply(CodeGenerator.scala:675) at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244) at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244) at scala.collection.Iterator$class.foreach(Iterator.scala:727) at scala.collection.AbstractIterator.foreach(Iterator.scala:1157) at scala.collection.IterableLike$class.foreach(IterableLike.scala:72) at scala.collection.AbstractIterable.foreach(Iterable.scala:54) at scala.collection.TraversableLike$class.map(TraversableLike.scala:244) at scala.collection.AbstractTraversable.map(Traversable.scala:105) at org.apache.flink.api.table.codegen.CodeGenerator.visitCall(CodeGenerator.scala:675) at org.apache.flink.api.table.codegen.CodeGenerator.visitCall(CodeGenerator.scala:56) at org.apache.calcite.rex.RexCall.accept(RexCall.java:108) at org.apache.flink.api.table.codegen.CodeGenerator$$anonfun$14.apply(CodeGenerator.scala:675) at org.apache.flink.api.table.codegen.CodeGenerator$$anonfun$14.apply(CodeGenerator.scala:675) at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244) at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244) at scala.collection.Iterator$class.foreach(Iterator.scala:727) at scala.collection.AbstractIterator.foreach(Iterator.scala:1157) at scala.collection.IterableLike$class.foreach(IterableLike.scala:72) at scala.collection.AbstractIterable.foreach(Iterable.scala:54) at scala.collection.TraversableLike$class.map(TraversableLike.scala:244) at scala.collection.AbstractTraversable.map(Traversable.scala:105) at org.apache.flink.api.table.codegen.CodeGenerator.visitCall(CodeGenerator.scala:675) at org.apache.flink.api.table.codegen.CodeGenerator.visitCall(CodeGenerator.scala:56) at org.apache.calcite.rex.RexCall.accept(RexCall.java:108) at org.apache.flink.api.table.codegen.CodeGenerator.generateExpression(CodeGenerator.scala:181) at org.apache.flink.api.table.codegen.CodeGenerator$$anonfun$5.apply(CodeGenerator.scala:300) at org.apache.flink.api.table.codegen.CodeGenerator$$anonfun$5.apply(CodeGenerator.scala:300) at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244) at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244) at scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59) at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:47) at scala.collection.TraversableLike$class.map(TraversableLike.scala:244) at scala.collection.AbstractTraversable.map(Traversable.scala:105) at org.apache.flink.api.table.codegen.CodeGenerator.generateResultExpression(CodeGenerator.scala:300) at org.apache.flink.api.table.plan.nodes.FlinkCalc$class.functionBody(FlinkCalc.scala:52) at org.apache.flink.api.table.plan.nodes.dataset.DataSetCalc.functionBody(DataSetCalc.scala:39) at org.apache.flink.api.table.plan.nodes.dataset.DataSetCalc.translateToPlan(DataSetCalc.scala:108) at org.apache.flink.api.table.BatchTableEnvironment.translate(BatchTableEnvironment.scala:274) at org.apache.flink.api.scala.table.BatchTableEnvironment.toDataSet(BatchTableEnvironment.scala:139) at org.apache.flink.api.scala.table.TableConversions.toDataSet(TableConversions.scala:41) at org.apache.flink.api.scala.batch.sql.AggregationsITCase.testVarSampAggregate(AggregationsITCase.scala:369) 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:497) at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:47) at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:44) at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17) 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:271) at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:70) at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:50) at org.junit.runners.ParentRunner$3.run(ParentRunner.java:238) at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:63) at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:236) at org.junit.runners.ParentRunner.access$000(ParentRunner.java:53) at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:229) at org.junit.runners.ParentRunner.run(ParentRunner.java:309) at org.junit.runners.Suite.runChild(Suite.java:127) at org.junit.runners.Suite.runChild(Suite.java:26) at org.junit.runners.ParentRunner$3.run(ParentRunner.java:238) at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:63) at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:236) at org.junit.runners.ParentRunner.access$000(ParentRunner.java:53) at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:229) 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.runners.ParentRunner.run(ParentRunner.java:309) at org.junit.runner.JUnitCore.run(JUnitCore.java:160) then I remove !AggregateReduceFunctionsRule.INSTANCE.matches(call) and return this code , tests is passed. I search resolution this problem, do you have any ideas about it?
          Hide
          twalthr Timo Walther added a comment -

          You cannot use !AggregateReduceFunctionsRule.INSTANCE.matches(call) as it is always false for sums. See AggregateReduceFunctionsRule.containsAvgStddevVarCall.

          What do you mean with "something went wrong"? What about this:

              val supported = agg.getAggCallList.map(_.getAggregation.getKind).forall {
                case SqlKind.SUM => true
                case SqlKind.MIN => true
                case SqlKind.MAX => true
                case _ => false
              }
          
              !distinctAggs && !groupSets && !agg.indicator && supported
          
          Show
          twalthr Timo Walther added a comment - You cannot use !AggregateReduceFunctionsRule.INSTANCE.matches(call) as it is always false for sums. See AggregateReduceFunctionsRule.containsAvgStddevVarCall . What do you mean with "something went wrong"? What about this: val supported = agg.getAggCallList.map(_.getAggregation.getKind).forall { case SqlKind.SUM => true case SqlKind.MIN => true case SqlKind.MAX => true case _ => false } !distinctAggs && !groupSets && !agg.indicator && supported
          Hide
          anmu Anton Mushin added a comment - - edited

          I tried use code like as your code and I got exception:

          org.apache.flink.api.table.TableException: Cannot generate a valid execution plan for the given query: 
          
          LogicalAggregate(group=[{}], EXPR$0=[STDDEV_POP($0)], EXPR$1=[STDDEV_POP($1)], EXPR$2=[STDDEV_POP($2)], EXPR$3=[STDDEV_POP($3)], EXPR$4=[STDDEV_POP($4)], EXPR$5=[STDDEV_POP($5)])
            LogicalTableScan(table=[[_DataSetTable_0]])
          
          This exception indicates that the query uses an unsupported SQL feature.
          Please check the documentation for the set of currently supported SQL features.
          
          	at org.apache.flink.api.table.BatchTableEnvironment.translate(BatchTableEnvironment.scala:257)
          	at org.apache.flink.api.scala.table.BatchTableEnvironment.toDataSet(BatchTableEnvironment.scala:139)
          	at org.apache.flink.api.scala.table.TableConversions.toDataSet(TableConversions.scala:41)
          	at org.apache.flink.api.scala.batch.sql.AggregationsITCase.testStddevPopAggregate(AggregationsITCase.scala:282)
          	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:497)
          	at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:47)
          	at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
          	at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:44)
          	at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17)
          	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:271)
          	at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:70)
          	at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:50)
          	at org.junit.runners.ParentRunner$3.run(ParentRunner.java:238)
          	at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:63)
          	at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:236)
          	at org.junit.runners.ParentRunner.access$000(ParentRunner.java:53)
          	at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:229)
          	at org.junit.runners.ParentRunner.run(ParentRunner.java:309)
          	at org.junit.runners.Suite.runChild(Suite.java:127)
          	at org.junit.runners.Suite.runChild(Suite.java:26)
          	at org.junit.runners.ParentRunner$3.run(ParentRunner.java:238)
          	at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:63)
          	at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:236)
          	at org.junit.runners.ParentRunner.access$000(ParentRunner.java:53)
          	at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:229)
          	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.runners.ParentRunner.run(ParentRunner.java:309)
          	at org.junit.runner.JUnitCore.run(JUnitCore.java:160)
          	at com.intellij.junit4.JUnit4IdeaTestRunner.startRunnerWithArgs(JUnit4IdeaTestRunner.java:69)
          	at com.intellij.rt.execution.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:234)
          	at com.intellij.rt.execution.junit.JUnitStarter.main(JUnitStarter.java:74)
          	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:497)
          	at com.intellij.rt.execution.application.AppMain.main(AppMain.java:144)
          

          You can look my code

          Show
          anmu Anton Mushin added a comment - - edited I tried use code like as your code and I got exception: org.apache.flink.api.table.TableException: Cannot generate a valid execution plan for the given query: LogicalAggregate(group=[{}], EXPR$0=[STDDEV_POP($0)], EXPR$1=[STDDEV_POP($1)], EXPR$2=[STDDEV_POP($2)], EXPR$3=[STDDEV_POP($3)], EXPR$4=[STDDEV_POP($4)], EXPR$5=[STDDEV_POP($5)]) LogicalTableScan(table=[[_DataSetTable_0]]) This exception indicates that the query uses an unsupported SQL feature. Please check the documentation for the set of currently supported SQL features. at org.apache.flink.api.table.BatchTableEnvironment.translate(BatchTableEnvironment.scala:257) at org.apache.flink.api.scala.table.BatchTableEnvironment.toDataSet(BatchTableEnvironment.scala:139) at org.apache.flink.api.scala.table.TableConversions.toDataSet(TableConversions.scala:41) at org.apache.flink.api.scala.batch.sql.AggregationsITCase.testStddevPopAggregate(AggregationsITCase.scala:282) 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:497) at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:47) at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:44) at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17) 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:271) at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:70) at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:50) at org.junit.runners.ParentRunner$3.run(ParentRunner.java:238) at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:63) at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:236) at org.junit.runners.ParentRunner.access$000(ParentRunner.java:53) at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:229) at org.junit.runners.ParentRunner.run(ParentRunner.java:309) at org.junit.runners.Suite.runChild(Suite.java:127) at org.junit.runners.Suite.runChild(Suite.java:26) at org.junit.runners.ParentRunner$3.run(ParentRunner.java:238) at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:63) at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:236) at org.junit.runners.ParentRunner.access$000(ParentRunner.java:53) at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:229) 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.runners.ParentRunner.run(ParentRunner.java:309) at org.junit.runner.JUnitCore.run(JUnitCore.java:160) at com.intellij.junit4.JUnit4IdeaTestRunner.startRunnerWithArgs(JUnit4IdeaTestRunner.java:69) at com.intellij.rt.execution.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:234) at com.intellij.rt.execution.junit.JUnitStarter.main(JUnitStarter.java:74) 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:497) at com.intellij.rt.execution.application.AppMain.main(AppMain.java:144) You can look my code
          Hide
          anmu Anton Mushin added a comment -

          Sorry, I will try explain my problem.
          I have two tests

          test1
          @Test
            def testStddevPopAggregate(): Unit = {
              val env = ExecutionEnvironment.getExecutionEnvironment
              val tEnv = TableEnvironment.getTableEnvironment(env, config)
              val ds = env.fromElements(
                (1: Byte, 1 : Short, 1, 1L, 1F, 1D),
                (2: Byte, 2 : Short, 2, 2L, 2F, 2D)).toTable(tEnv)
              tEnv.registerTable("myTable", ds)
              val columns = Array("_1","_2","_3","_4","_5","_6")
          
              val sqlQuery = getSelectQuery("STDDEV_POP(?)")(columns,"myTable")
              //val sqlExpectedQuery = getSelectQuery("SQRT((SUM(? * ?) - SUM(?) * SUM(?) / COUNT(?)) / COUNT(?))")(columns,"myTable")
          
              val actualResult = tEnv.sql(sqlQuery).toDataSet[Row].collect()
              //val expectedResult = tEnv.sql(sqlExpectedQuery).toDataSet[Row].collect().toString.replaceAll("Buffer\\(|\\)", "")
              val expectedResult = "0,0,0,0,0.5,0.5"
              TestBaseUtils.compareOrderedResultAsText(actualResult.asJava, expectedResult)
            }
          
          test2
          @Test
            def testStddevPopAggregateWithOtherAggreagte(): Unit = {
              val localconf = config
              localconf.setNullCheck(true)
              val env = ExecutionEnvironment.getExecutionEnvironment
              val tEnv = TableEnvironment.getTableEnvironment(env, localconf)
          
              val ds = env.fromElements(
                (1: Byte, 1 : Short, 1, 1L, 1F, 1D),
                (2: Byte, 2 : Short, 2, 2L, 2F, 2D)).toTable(tEnv)
              tEnv.registerTable("myTable", ds)
              val columns = Array("_1","_2","_3","_4","_5","_6")
          
              val sqlQuery = getSelectQuery("STDDEV_POP(?), avg(?), sum(?), max(?), min(?), count(?)")(columns,"myTable")
          
              val sqlExpectedQuery = getSelectQuery("SQRT((SUM(? * ?) - SUM(?) * SUM(?) / COUNT(?)) / COUNT(?))" +
                "avg(?),sum(?),max(?),min(?),count(?)")(columns,"myTable")
          
              val actualResult = tEnv.sql(sqlQuery).toDataSet[Row].collect()
              val expectedResult = tEnv.sql(sqlExpectedQuery).toDataSet[Row].collect().toString.replaceAll("Buffer\\(|\\)", "")
              //val expectedResult = "0.0,1,3,2,1,2,0.0,1,3,2,1,2,0.0,1,3,2,1,2,0.0,1,3,2,1,2,0.5,1.5,3.0,2.0,1.0,2,0.5,1.5,3.0,2.0,1.0,2"
              TestBaseUtils.compareOrderedResultAsText(actualResult.asJava, expectedResult)
            }
          

          Actual code for running tests.
          First test is passed and second isn't.
          I have Calcite ST:

          org.apache.calcite.plan.RelOptPlanner$CannotPlanException: Node [rel#7:Subset#1.DATASET.[]] could not be implemented; planner state:
          
          Root: rel#7:Subset#1.DATASET.[]
          Original rel:
          
          Sets:
          Set#0, type: RecordType(TINYINT _1, SMALLINT _2, INTEGER _3, BIGINT _4, FLOAT _5, DOUBLE _6)
          	rel#4:Subset#0.NONE.[], best=null, importance=0.81
          		rel#0:LogicalTableScan.NONE.[](table=[_DataSetTable_0]), rowcount=1000.0, cumulative cost={inf}
          	rel#26:Subset#0.DATASET.[], best=rel#25, importance=0.405
          		rel#25:DataSetScan.DATASET.[](table=[_DataSetTable_0]), rowcount=1000.0, cumulative cost={1000.0 rows, 1000.0 cpu, 0.0 io}
          	rel#28:Subset#0.ENUMERABLE.[], best=rel#27, importance=0.405
          		rel#27:EnumerableTableScan.ENUMERABLE.[](table=[_DataSetTable_0]), rowcount=1000.0, cumulative cost={1000.0 rows, 1001.0 cpu, 0.0 io}
          Set#1, type: RecordType(TINYINT EXPR$0, TINYINT EXPR$1, TINYINT EXPR$2, TINYINT EXPR$3, TINYINT EXPR$4, BIGINT EXPR$5, SMALLINT EXPR$6, SMALLINT EXPR$7, SMALLINT EXPR$8, SMALLINT EXPR$9, SMALLINT EXPR$10, BIGINT EXPR$11, INTEGER EXPR$12, INTEGER EXPR$13, INTEGER EXPR$14, INTEGER EXPR$15, INTEGER EXPR$16, BIGINT EXPR$17, BIGINT EXPR$18, BIGINT EXPR$19, BIGINT EXPR$20, BIGINT EXPR$21, BIGINT EXPR$22, BIGINT EXPR$23, FLOAT EXPR$24, FLOAT EXPR$25, FLOAT EXPR$26, FLOAT EXPR$27, FLOAT EXPR$28, BIGINT EXPR$29, DOUBLE EXPR$30, DOUBLE EXPR$31, DOUBLE EXPR$32, DOUBLE EXPR$33, DOUBLE EXPR$34, BIGINT EXPR$35)
          	rel#6:Subset#1.NONE.[], best=null, importance=0.9
          		rel#5:LogicalAggregate.NONE.[](input=rel#4:Subset#0.NONE.[],group={},EXPR$0=STDDEV_POP($0),EXPR$1=AVG($0),EXPR$2=SUM($0),EXPR$3=MAX($0),EXPR$4=MIN($0),EXPR$5=COUNT($0),EXPR$6=STDDEV_POP($1),EXPR$7=AVG($1),EXPR$8=SUM($1),EXPR$9=MAX($1),EXPR$10=MIN($1),EXPR$11=COUNT($1),EXPR$12=STDDEV_POP($2),EXPR$13=AVG($2),EXPR$14=SUM($2),EXPR$15=MAX($2),EXPR$16=MIN($2),EXPR$17=COUNT($2),EXPR$18=STDDEV_POP($3),EXPR$19=AVG($3),EXPR$20=SUM($3),EXPR$21=MAX($3),EXPR$22=MIN($3),EXPR$23=COUNT($3),EXPR$24=STDDEV_POP($4),EXPR$25=AVG($4),EXPR$26=SUM($4),EXPR$27=MAX($4),EXPR$28=MIN($4),EXPR$29=COUNT($4),EXPR$30=STDDEV_POP($5),EXPR$31=AVG($5),EXPR$32=SUM($5),EXPR$33=MAX($5),EXPR$34=MIN($5),EXPR$35=COUNT($5)), rowcount=100.0, cumulative cost={inf}
          		rel#15:LogicalProject.NONE.[](input=rel#14:Subset#3.NONE.[],EXPR$0=CAST(POWER(/(-($0, /(*($1, $1), $2)), $2), 0.5)):TINYINT,EXPR$1=CAST(/($1, $2)):TINYINT,EXPR$2=CASE(=($2, 0), null, $3),EXPR$3=$4,EXPR$4=$5,EXPR$5=$2,EXPR$6=CAST(POWER(/(-($6, /(*($7, $7), $8)), $8), 0.5)):SMALLINT,EXPR$7=CAST(/($7, $8)):SMALLINT,EXPR$8=CASE(=($8, 0), null, $9),EXPR$9=$10,EXPR$10=$11,EXPR$11=$8,EXPR$12=CAST(POWER(/(-($12, /(*($13, $13), $14)), $14), 0.5)):INTEGER,EXPR$13=CAST(/($13, $14)):INTEGER,EXPR$14=CASE(=($14, 0), null, $15),EXPR$15=$16,EXPR$16=$17,EXPR$17=$14,EXPR$18=CAST(POWER(/(-($18, /(*($19, $19), $20)), $20), 0.5)):BIGINT,EXPR$19=CAST(/($19, $20)):BIGINT,EXPR$20=CASE(=($20, 0), null, $21),EXPR$21=$22,EXPR$22=$23,EXPR$23=$20,EXPR$24=CAST(POWER(/(-($24, /(*($25, $25), $26)), $26), 0.5)):FLOAT,EXPR$25=CAST(/($25, $26)):FLOAT,EXPR$26=CASE(=($26, 0), null, $27),EXPR$27=$28,EXPR$28=$29,EXPR$29=$26,EXPR$30=CAST(POWER(/(-($30, /(*($31, $31), $32)), $32), 0.5)):DOUBLE,EXPR$31=CAST(/($31, $32)):DOUBLE,EXPR$32=CASE(=($32, 0), null, $33),EXPR$33=$34,EXPR$34=$35,EXPR$35=$32), rowcount=100.0, cumulative cost={inf}
          		rel#16:LogicalCalc.NONE.[[]](input=rel#14:Subset#3.NONE.[],expr#0..35={inputs},expr#36=*($t1, $t1),expr#37=/($t36, $t2),expr#38=-($t0, $t37),expr#39=/($t38, $t2),expr#40=0.5,expr#41=POWER($t39, $t40),expr#42=CAST($t41):TINYINT,expr#43=/($t1, $t2),expr#44=CAST($t43):TINYINT,expr#45=0,expr#46==($t2, $t45),expr#47=null,expr#48=CASE($t46, $t47, $t3),expr#49=*($t7, $t7),expr#50=/($t49, $t8),expr#51=-($t6, $t50),expr#52=/($t51, $t8),expr#53=POWER($t52, $t40),expr#54=CAST($t53):SMALLINT,expr#55=/($t7, $t8),expr#56=CAST($t55):SMALLINT,expr#57==($t8, $t45),expr#58=CASE($t57, $t47, $t9),expr#59=*($t13, $t13),expr#60=/($t59, $t14),expr#61=-($t12, $t60),expr#62=/($t61, $t14),expr#63=POWER($t62, $t40),expr#64=CAST($t63):INTEGER,expr#65=/($t13, $t14),expr#66=CAST($t65):INTEGER,expr#67==($t14, $t45),expr#68=CASE($t67, $t47, $t15),expr#69=*($t19, $t19),expr#70=/($t69, $t20),expr#71=-($t18, $t70),expr#72=/($t71, $t20),expr#73=POWER($t72, $t40),expr#74=CAST($t73):BIGINT,expr#75=/($t19, $t20),expr#76=CAST($t75):BIGINT,expr#77==($t20, $t45),expr#78=CASE($t77, $t47, $t21),expr#79=*($t25, $t25),expr#80=/($t79, $t26),expr#81=-($t24, $t80),expr#82=/($t81, $t26),expr#83=POWER($t82, $t40),expr#84=CAST($t83):FLOAT,expr#85=/($t25, $t26),expr#86=CAST($t85):FLOAT,expr#87==($t26, $t45),expr#88=CASE($t87, $t47, $t27),expr#89=*($t31, $t31),expr#90=/($t89, $t32),expr#91=-($t30, $t90),expr#92=/($t91, $t32),expr#93=POWER($t92, $t40),expr#94=CAST($t93):DOUBLE,expr#95=/($t31, $t32),expr#96=CAST($t95):DOUBLE,expr#97==($t32, $t45),expr#98=CASE($t97, $t47, $t33),EXPR$0=$t42,EXPR$1=$t44,EXPR$2=$t48,EXPR$3=$t4,EXPR$4=$t5,EXPR$5=$t2,EXPR$6=$t54,EXPR$7=$t56,EXPR$8=$t58,EXPR$9=$t10,EXPR$10=$t11,EXPR$11=$t8,EXPR$12=$t64,EXPR$13=$t66,EXPR$14=$t68,EXPR$15=$t16,EXPR$16=$t17,EXPR$17=$t14,EXPR$18=$t74,EXPR$19=$t76,EXPR$20=$t78,EXPR$21=$t22,EXPR$22=$t23,EXPR$23=$t20,EXPR$24=$t84,EXPR$25=$t86,EXPR$26=$t88,EXPR$27=$t28,EXPR$28=$t29,EXPR$29=$t26,EXPR$30=$t94,EXPR$31=$t96,EXPR$32=$t98,EXPR$33=$t34,EXPR$34=$t35,EXPR$35=$t32), importance=0.0, rowcount=100.0, cumulative cost={inf}
          		rel#19:LogicalCalc.NONE.[[]](input=rel#14:Subset#3.NONE.[],expr#0..35={inputs},expr#36=*($t1, $t1),expr#37=/($t36, $t2),expr#38=-($t0, $t37),expr#39=/($t38, $t2),expr#40=0.5,expr#41=POWER($t39, $t40),expr#42=CAST($t41):TINYINT,expr#43=/($t1, $t2),expr#44=CAST($t43):TINYINT,expr#45=0,expr#46==($t2, $t45),expr#47=null,expr#48=CASE($t46, $t47, $t3),expr#49=*($t7, $t7),expr#50=/($t49, $t8),expr#51=-($t6, $t50),expr#52=/($t51, $t8),expr#53=POWER($t52, $t40),expr#54=CAST($t53):SMALLINT,expr#55=/($t7, $t8),expr#56=CAST($t55):SMALLINT,expr#57==($t8, $t45),expr#58=CASE($t57, $t47, $t9),expr#59=*($t13, $t13),expr#60=/($t59, $t14),expr#61=-($t12, $t60),expr#62=/($t61, $t14),expr#63=POWER($t62, $t40),expr#64=CAST($t63):INTEGER,expr#65=/($t13, $t14),expr#66=CAST($t65):INTEGER,expr#67==($t14, $t45),expr#68=CASE($t67, $t47, $t15),expr#69=*($t19, $t19),expr#70=/($t69, $t20),expr#71=-($t18, $t70),expr#72=/($t71, $t20),expr#73=POWER($t72, $t40),expr#74=CAST($t73):BIGINT,expr#75=/($t19, $t20),expr#76==($t20, $t45),expr#77=CASE($t76, $t47, $t21),expr#78=*($t25, $t25),expr#79=/($t78, $t26),expr#80=-($t24, $t79),expr#81=/($t80, $t26),expr#82=POWER($t81, $t40),expr#83=CAST($t82):FLOAT,expr#84=/($t25, $t26),expr#85==($t26, $t45),expr#86=CASE($t85, $t47, $t27),expr#87=*($t31, $t31),expr#88=/($t87, $t32),expr#89=-($t30, $t88),expr#90=/($t89, $t32),expr#91=POWER($t90, $t40),expr#92=/($t31, $t32),expr#93==($t32, $t45),expr#94=CASE($t93, $t47, $t33),EXPR$0=$t42,EXPR$1=$t44,EXPR$2=$t48,EXPR$3=$t4,EXPR$4=$t5,EXPR$5=$t2,EXPR$6=$t54,EXPR$7=$t56,EXPR$8=$t58,EXPR$9=$t10,EXPR$10=$t11,EXPR$11=$t8,EXPR$12=$t64,EXPR$13=$t66,EXPR$14=$t68,EXPR$15=$t16,EXPR$16=$t17,EXPR$17=$t14,EXPR$18=$t74,EXPR$19=$t75,EXPR$20=$t77,EXPR$21=$t22,EXPR$22=$t23,EXPR$23=$t20,EXPR$24=$t83,EXPR$25=$t84,EXPR$26=$t86,EXPR$27=$t28,EXPR$28=$t29,EXPR$29=$t26,EXPR$30=$t91,EXPR$31=$t92,EXPR$32=$t94,EXPR$33=$t34,EXPR$34=$t35,EXPR$35=$t32), rowcount=100.0, cumulative cost={inf}
          		rel#31:LogicalCalc.NONE.[[]](input=rel#23:Subset#4.NONE.[],expr#0..35={inputs},expr#36=0,expr#37==($t1, $t36),expr#38=null,expr#39=CASE($t37, $t38, $t0),expr#40==($t3, $t36),expr#41=CASE($t40, $t38, $t2),expr#42=*($t41, $t41),expr#43=/($t42, $t3),expr#44=-($t39, $t43),expr#45=/($t44, $t3),expr#46=0.5,expr#47=POWER($t45, $t46),expr#48=CAST($t47):TINYINT,expr#49=/($t41, $t3),expr#50=CAST($t49):TINYINT,expr#51==($t7, $t36),expr#52=CASE($t51, $t38, $t6),expr#53==($t9, $t36),expr#54=CASE($t53, $t38, $t8),expr#55=*($t54, $t54),expr#56=/($t55, $t9),expr#57=-($t52, $t56),expr#58=/($t57, $t9),expr#59=POWER($t58, $t46),expr#60=CAST($t59):SMALLINT,expr#61=/($t54, $t9),expr#62=CAST($t61):SMALLINT,expr#63==($t13, $t36),expr#64=CASE($t63, $t38, $t12),expr#65==($t15, $t36),expr#66=CASE($t65, $t38, $t14),expr#67=*($t66, $t66),expr#68=/($t67, $t15),expr#69=-($t64, $t68),expr#70=/($t69, $t15),expr#71=POWER($t70, $t46),expr#72=CAST($t71):INTEGER,expr#73=/($t66, $t15),expr#74=CAST($t73):INTEGER,expr#75==($t19, $t36),expr#76=CASE($t75, $t38, $t18),expr#77==($t21, $t36),expr#78=CASE($t77, $t38, $t20),expr#79=*($t78, $t78),expr#80=/($t79, $t21),expr#81=-($t76, $t80),expr#82=/($t81, $t21),expr#83=POWER($t82, $t46),expr#84=CAST($t83):BIGINT,expr#85=/($t78, $t21),expr#86=CAST($t85):BIGINT,expr#87==($t25, $t36),expr#88=CASE($t87, $t38, $t24),expr#89==($t27, $t36),expr#90=CASE($t89, $t38, $t26),expr#91=*($t90, $t90),expr#92=/($t91, $t27),expr#93=-($t88, $t92),expr#94=/($t93, $t27),expr#95=POWER($t94, $t46),expr#96=CAST($t95):FLOAT,expr#97=/($t90, $t27),expr#98=CAST($t97):FLOAT,expr#99==($t31, $t36),expr#100=CASE($t99, $t38, $t30),expr#101==($t33, $t36),expr#102=CASE($t101, $t38, $t32),expr#103=*($t102, $t102),expr#104=/($t103, $t33),expr#105=-($t100, $t104),expr#106=/($t105, $t33),expr#107=POWER($t106, $t46),expr#108=CAST($t107):DOUBLE,expr#109=/($t102, $t33),expr#110=CAST($t109):DOUBLE,EXPR$0=$t48,EXPR$1=$t50,EXPR$2=$t41,EXPR$3=$t4,EXPR$4=$t5,EXPR$5=$t3,EXPR$6=$t60,EXPR$7=$t62,EXPR$8=$t54,EXPR$9=$t10,EXPR$10=$t11,EXPR$11=$t9,EXPR$12=$t72,EXPR$13=$t74,EXPR$14=$t66,EXPR$15=$t16,EXPR$16=$t17,EXPR$17=$t15,EXPR$18=$t84,EXPR$19=$t86,EXPR$20=$t78,EXPR$21=$t22,EXPR$22=$t23,EXPR$23=$t21,EXPR$24=$t96,EXPR$25=$t98,EXPR$26=$t90,EXPR$27=$t28,EXPR$28=$t29,EXPR$29=$t27,EXPR$30=$t108,EXPR$31=$t110,EXPR$32=$t102,EXPR$33=$t34,EXPR$34=$t35,EXPR$35=$t33), importance=0.0, rowcount=100.0, cumulative cost={inf}
          		rel#34:LogicalCalc.NONE.[[]](input=rel#23:Subset#4.NONE.[],expr#0..35={inputs},expr#36=0,expr#37==($t1, $t36),expr#38=null,expr#39=CASE($t37, $t38, $t0),expr#40==($t3, $t36),expr#41=CASE($t40, $t38, $t2),expr#42=*($t41, $t41),expr#43=/($t42, $t3),expr#44=-($t39, $t43),expr#45=/($t44, $t3),expr#46=0.5,expr#47=POWER($t45, $t46),expr#48=CAST($t47):TINYINT,expr#49=/($t41, $t3),expr#50=CAST($t49):TINYINT,expr#51==($t7, $t36),expr#52=CASE($t51, $t38, $t6),expr#53==($t9, $t36),expr#54=CASE($t53, $t38, $t8),expr#55=*($t54, $t54),expr#56=/($t55, $t9),expr#57=-($t52, $t56),expr#58=/($t57, $t9),expr#59=POWER($t58, $t46),expr#60=CAST($t59):SMALLINT,expr#61=/($t54, $t9),expr#62=CAST($t61):SMALLINT,expr#63==($t13, $t36),expr#64=CASE($t63, $t38, $t12),expr#65==($t15, $t36),expr#66=CASE($t65, $t38, $t14),expr#67=*($t66, $t66),expr#68=/($t67, $t15),expr#69=-($t64, $t68),expr#70=/($t69, $t15),expr#71=POWER($t70, $t46),expr#72=CAST($t71):INTEGER,expr#73=/($t66, $t15),expr#74=CAST($t73):INTEGER,expr#75==($t19, $t36),expr#76=CASE($t75, $t38, $t18),expr#77==($t21, $t36),expr#78=CASE($t77, $t38, $t20),expr#79=*($t78, $t78),expr#80=/($t79, $t21),expr#81=-($t76, $t80),expr#82=/($t81, $t21),expr#83=POWER($t82, $t46),expr#84=CAST($t83):BIGINT,expr#85=/($t78, $t21),expr#86==($t25, $t36),expr#87=CASE($t86, $t38, $t24),expr#88==($t27, $t36),expr#89=CASE($t88, $t38, $t26),expr#90=*($t89, $t89),expr#91=/($t90, $t27),expr#92=-($t87, $t91),expr#93=/($t92, $t27),expr#94=POWER($t93, $t46),expr#95=CAST($t94):FLOAT,expr#96=/($t89, $t27),expr#97==($t31, $t36),expr#98=CASE($t97, $t38, $t30),expr#99==($t33, $t36),expr#100=CASE($t99, $t38, $t32),expr#101=*($t100, $t100),expr#102=/($t101, $t33),expr#103=-($t98, $t102),expr#104=/($t103, $t33),expr#105=POWER($t104, $t46),expr#106=/($t100, $t33),EXPR$0=$t48,EXPR$1=$t50,EXPR$2=$t41,EXPR$3=$t4,EXPR$4=$t5,EXPR$5=$t3,EXPR$6=$t60,EXPR$7=$t62,EXPR$8=$t54,EXPR$9=$t10,EXPR$10=$t11,EXPR$11=$t9,EXPR$12=$t72,EXPR$13=$t74,EXPR$14=$t66,EXPR$15=$t16,EXPR$16=$t17,EXPR$17=$t15,EXPR$18=$t84,EXPR$19=$t85,EXPR$20=$t78,EXPR$21=$t22,EXPR$22=$t23,EXPR$23=$t21,EXPR$24=$t95,EXPR$25=$t96,EXPR$26=$t89,EXPR$27=$t28,EXPR$28=$t29,EXPR$29=$t27,EXPR$30=$t105,EXPR$31=$t106,EXPR$32=$t100,EXPR$33=$t34,EXPR$34=$t35,EXPR$35=$t33), rowcount=100.0, cumulative cost={inf}
          	rel#7:Subset#1.DATASET.[], best=null, importance=1.0
          		rel#8:AbstractConverter.DATASET.[](input=rel#6:Subset#1.NONE.[],convention=DATASET,sort=[]), rowcount=100.0, cumulative cost={inf}
          		rel#18:DataSetCalc.DATASET.[[]](input=rel#17:Subset#3.DATASET.[],select=CAST(POWER(/(-($f0, /(*($f1, $f1), $f2)), $f2), 0.5)) AS $f0, CAST(/($f1, $f2)) AS $f1, CASE(=($f2, 0), null, EXPR$2) AS $f2, EXPR$3 AS EXPR$2, EXPR$4 AS EXPR$3, $f2 AS EXPR$4, CAST(POWER(/(-($f6, /(*($f7, $f7), $f8)), $f8), 0.5)) AS $f6, CAST(/($f7, $f8)) AS $f7, CASE(=($f8, 0), null, EXPR$8) AS $f8, EXPR$9 AS EXPR$8, EXPR$10 AS EXPR$9, $f8 AS EXPR$10, CAST(POWER(/(-($f12, /(*($f13, $f13), $f14)), $f14), 0.5)) AS $f12, CAST(/($f13, $f14)) AS $f13, CASE(=($f14, 0), null, EXPR$14) AS $f14, EXPR$15 AS EXPR$14, EXPR$16 AS EXPR$15, $f14 AS EXPR$16, CAST(POWER(/(-($f18, /(*($f19, $f19), $f20)), $f20), 0.5)) AS $f18, CAST(/($f19, $f20)) AS $f19, CASE(=($f20, 0), null, EXPR$20) AS $f20, EXPR$21 AS EXPR$20, EXPR$22 AS EXPR$21, $f20 AS EXPR$22, CAST(POWER(/(-($f24, /(*($f25, $f25), $f26)), $f26), 0.5)) AS $f24, CAST(/($f25, $f26)) AS $f25, CASE(=($f26, 0), null, EXPR$26) AS $f26, EXPR$27 AS EXPR$26, EXPR$28 AS EXPR$27, $f26 AS EXPR$28, CAST(POWER(/(-($f30, /(*($f31, $f31), $f32)), $f32), 0.5)) AS $f30, CAST(/($f31, $f32)) AS $f31, CASE(=($f32, 0), null, EXPR$32) AS $f32, EXPR$33 AS EXPR$32, EXPR$34 AS EXPR$33, $f32 AS EXPR$34), rowcount=100.0, cumulative cost={inf}
          		rel#20:DataSetCalc.DATASET.[[]](input=rel#17:Subset#3.DATASET.[],select=CAST(POWER(/(-($f0, /(*($f1, $f1), $f2)), $f2), 0.5)) AS $f0, CAST(/($f1, $f2)) AS $f1, CASE(=($f2, 0), null, EXPR$2) AS $f2, EXPR$3 AS EXPR$2, EXPR$4 AS EXPR$3, $f2 AS EXPR$4, CAST(POWER(/(-($f6, /(*($f7, $f7), $f8)), $f8), 0.5)) AS $f6, CAST(/($f7, $f8)) AS $f7, CASE(=($f8, 0), null, EXPR$8) AS $f8, EXPR$9 AS EXPR$8, EXPR$10 AS EXPR$9, $f8 AS EXPR$10, CAST(POWER(/(-($f12, /(*($f13, $f13), $f14)), $f14), 0.5)) AS $f12, CAST(/($f13, $f14)) AS $f13, CASE(=($f14, 0), null, EXPR$14) AS $f14, EXPR$15 AS EXPR$14, EXPR$16 AS EXPR$15, $f14 AS EXPR$16, CAST(POWER(/(-($f18, /(*($f19, $f19), $f20)), $f20), 0.5)) AS $f18, /($f19, $f20) AS $f19, CASE(=($f20, 0), null, EXPR$20) AS $f20, EXPR$21 AS EXPR$20, EXPR$22 AS EXPR$21, $f20 AS EXPR$22, CAST(POWER(/(-($f24, /(*($f25, $f25), $f26)), $f26), 0.5)) AS $f24, /($f25, $f26) AS $f25, CASE(=($f26, 0), null, EXPR$26) AS $f26, EXPR$27 AS EXPR$26, EXPR$28 AS EXPR$27, $f26 AS EXPR$28, POWER(/(-($f30, /(*($f31, $f31), $f32)), $f32), 0.5) AS $f30, /($f31, $f32) AS $f31, CASE(=($f32, 0), null, EXPR$32) AS $f32, EXPR$33 AS EXPR$32, EXPR$34 AS EXPR$33, $f32 AS EXPR$34), rowcount=100.0, cumulative cost={inf}
          		rel#33:DataSetCalc.DATASET.[[]](input=rel#32:Subset#4.DATASET.[],select=CAST(POWER(/(-(CASE(=($f1, 0), null, $f0), /(*(CASE(=($f3, 0), null, $f2), CASE(=($f3, 0), null, $f2)), $f3)), $f3), 0.5)) AS $f0, CAST(/(CASE(=($f3, 0), null, $f2), $f3)) AS $f1, CASE(=($f3, 0), null, $f2) AS $f2, EXPR$3 AS $f3, EXPR$4 AS EXPR$3, $f3 AS EXPR$4, CAST(POWER(/(-(CASE(=($f7, 0), null, $f6), /(*(CASE(=($f9, 0), null, $f8), CASE(=($f9, 0), null, $f8)), $f9)), $f9), 0.5)) AS $f6, CAST(/(CASE(=($f9, 0), null, $f8), $f9)) AS $f7, CASE(=($f9, 0), null, $f8) AS $f8, EXPR$9 AS $f9, EXPR$10 AS EXPR$9, $f9 AS EXPR$10, CAST(POWER(/(-(CASE(=($f13, 0), null, $f12), /(*(CASE(=($f15, 0), null, $f14), CASE(=($f15, 0), null, $f14)), $f15)), $f15), 0.5)) AS $f12, CAST(/(CASE(=($f15, 0), null, $f14), $f15)) AS $f13, CASE(=($f15, 0), null, $f14) AS $f14, EXPR$15 AS $f15, EXPR$16 AS EXPR$15, $f15 AS EXPR$16, CAST(POWER(/(-(CASE(=($f19, 0), null, $f18), /(*(CASE(=($f21, 0), null, $f20), CASE(=($f21, 0), null, $f20)), $f21)), $f21), 0.5)) AS $f18, CAST(/(CASE(=($f21, 0), null, $f20), $f21)) AS $f19, CASE(=($f21, 0), null, $f20) AS $f20, EXPR$21 AS $f21, EXPR$22 AS EXPR$21, $f21 AS EXPR$22, CAST(POWER(/(-(CASE(=($f25, 0), null, $f24), /(*(CASE(=($f27, 0), null, $f26), CASE(=($f27, 0), null, $f26)), $f27)), $f27), 0.5)) AS $f24, CAST(/(CASE(=($f27, 0), null, $f26), $f27)) AS $f25, CASE(=($f27, 0), null, $f26) AS $f26, EXPR$27 AS $f27, EXPR$28 AS EXPR$27, $f27 AS EXPR$28, CAST(POWER(/(-(CASE(=($f31, 0), null, $f30), /(*(CASE(=($f33, 0), null, $f32), CASE(=($f33, 0), null, $f32)), $f33)), $f33), 0.5)) AS $f30, CAST(/(CASE(=($f33, 0), null, $f32), $f33)) AS $f31, CASE(=($f33, 0), null, $f32) AS $f32, EXPR$33 AS $f33, EXPR$34 AS EXPR$33, $f33 AS EXPR$34), rowcount=100.0, cumulative cost={inf}
          		rel#35:DataSetCalc.DATASET.[[]](input=rel#32:Subset#4.DATASET.[],select=CAST(POWER(/(-(CASE(=($f1, 0), null, $f0), /(*(CASE(=($f3, 0), null, $f2), CASE(=($f3, 0), null, $f2)), $f3)), $f3), 0.5)) AS $f0, CAST(/(CASE(=($f3, 0), null, $f2), $f3)) AS $f1, CASE(=($f3, 0), null, $f2) AS $f2, EXPR$3 AS $f3, EXPR$4 AS EXPR$3, $f3 AS EXPR$4, CAST(POWER(/(-(CASE(=($f7, 0), null, $f6), /(*(CASE(=($f9, 0), null, $f8), CASE(=($f9, 0), null, $f8)), $f9)), $f9), 0.5)) AS $f6, CAST(/(CASE(=($f9, 0), null, $f8), $f9)) AS $f7, CASE(=($f9, 0), null, $f8) AS $f8, EXPR$9 AS $f9, EXPR$10 AS EXPR$9, $f9 AS EXPR$10, CAST(POWER(/(-(CASE(=($f13, 0), null, $f12), /(*(CASE(=($f15, 0), null, $f14), CASE(=($f15, 0), null, $f14)), $f15)), $f15), 0.5)) AS $f12, CAST(/(CASE(=($f15, 0), null, $f14), $f15)) AS $f13, CASE(=($f15, 0), null, $f14) AS $f14, EXPR$15 AS $f15, EXPR$16 AS EXPR$15, $f15 AS EXPR$16, CAST(POWER(/(-(CASE(=($f19, 0), null, $f18), /(*(CASE(=($f21, 0), null, $f20), CASE(=($f21, 0), null, $f20)), $f21)), $f21), 0.5)) AS $f18, /(CASE(=($f21, 0), null, $f20), $f21) AS $f19, CASE(=($f21, 0), null, $f20) AS $f20, EXPR$21 AS $f21, EXPR$22 AS EXPR$21, $f21 AS EXPR$22, CAST(POWER(/(-(CASE(=($f25, 0), null, $f24), /(*(CASE(=($f27, 0), null, $f26), CASE(=($f27, 0), null, $f26)), $f27)), $f27), 0.5)) AS $f24, /(CASE(=($f27, 0), null, $f26), $f27) AS $f25, CASE(=($f27, 0), null, $f26) AS $f26, EXPR$27 AS $f27, EXPR$28 AS EXPR$27, $f27 AS EXPR$28, POWER(/(-(CASE(=($f31, 0), null, $f30), /(*(CASE(=($f33, 0), null, $f32), CASE(=($f33, 0), null, $f32)), $f33)), $f33), 0.5) AS $f30, /(CASE(=($f33, 0), null, $f32), $f33) AS $f31, CASE(=($f33, 0), null, $f32) AS $f32, EXPR$33 AS $f33, EXPR$34 AS EXPR$33, $f33 AS EXPR$34), rowcount=100.0, cumulative cost={inf}
          Set#2, type: RecordType(TINYINT _1, SMALLINT _2, INTEGER _3, BIGINT _4, FLOAT _5, DOUBLE _6, TINYINT $f6, SMALLINT $f7, INTEGER $f8, BIGINT $f9, FLOAT $f10, DOUBLE $f11)
          	rel#12:Subset#2.NONE.[], best=null, importance=0.7290000000000001
          		rel#9:LogicalProject.NONE.[](input=rel#4:Subset#0.NONE.[],_1=$0,_2=$1,_3=$2,_4=$3,_5=$4,_6=$5,$f6=*($0, $0),$f7=*($1, $1),$f8=*($2, $2),$f9=*($3, $3),$f10=*($4, $4),$f11=*($5, $5)), rowcount=1000.0, cumulative cost={inf}
          		rel#29:LogicalCalc.NONE.[[]](input=rel#4:Subset#0.NONE.[],expr#0..5={inputs},expr#6=*($t0, $t0),expr#7=*($t1, $t1),expr#8=*($t2, $t2),expr#9=*($t3, $t3),expr#10=*($t4, $t4),expr#11=*($t5, $t5),_1=$t0,_2=$t1,_3=$t2,_4=$t3,_5=$t4,_6=$t5,$f6=$t6,$f7=$t7,$f8=$t8,$f9=$t9,$f10=$t10,$f11=$t11), rowcount=1000.0, cumulative cost={inf}
          	rel#38:Subset#2.DATASET.[], best=rel#37, importance=0.36450000000000005
          		rel#37:DataSetCalc.DATASET.[[]](input=rel#26:Subset#0.DATASET.[],select=_1, _2, _3, _4, _5, _6), rowcount=1000.0, cumulative cost={2000.0 rows, 25000.0 cpu, 0.0 io}
          Set#3, type: RecordType(TINYINT $f0, TINYINT $f1, BIGINT $f2, TINYINT EXPR$2, TINYINT EXPR$3, TINYINT EXPR$4, SMALLINT $f6, SMALLINT $f7, BIGINT $f8, SMALLINT EXPR$8, SMALLINT EXPR$9, SMALLINT EXPR$10, INTEGER $f12, INTEGER $f13, BIGINT $f14, INTEGER EXPR$14, INTEGER EXPR$15, INTEGER EXPR$16, BIGINT $f18, BIGINT $f19, BIGINT $f20, BIGINT EXPR$20, BIGINT EXPR$21, BIGINT EXPR$22, FLOAT $f24, FLOAT $f25, BIGINT $f26, FLOAT EXPR$26, FLOAT EXPR$27, FLOAT EXPR$28, DOUBLE $f30, DOUBLE $f31, BIGINT $f32, DOUBLE EXPR$32, DOUBLE EXPR$33, DOUBLE EXPR$34)
          	rel#14:Subset#3.NONE.[], best=null, importance=0.81
          		rel#13:LogicalAggregate.NONE.[](input=rel#12:Subset#2.NONE.[],group={},agg#0=SUM($6),agg#1=SUM($0),agg#2=COUNT($0),EXPR$2=$SUM0($0),EXPR$3=MAX($0),EXPR$4=MIN($0),agg#6=SUM($7),agg#7=SUM($1),agg#8=COUNT($1),EXPR$8=$SUM0($1),EXPR$9=MAX($1),EXPR$10=MIN($1),agg#12=SUM($8),agg#13=SUM($2),agg#14=COUNT($2),EXPR$14=$SUM0($2),EXPR$15=MAX($2),EXPR$16=MIN($2),agg#18=SUM($9),agg#19=SUM($3),agg#20=COUNT($3),EXPR$20=$SUM0($3),EXPR$21=MAX($3),EXPR$22=MIN($3),agg#24=SUM($10),agg#25=SUM($4),agg#26=COUNT($4),EXPR$26=$SUM0($4),EXPR$27=MAX($4),EXPR$28=MIN($4),agg#30=SUM($11),agg#31=SUM($5),agg#32=COUNT($5),EXPR$32=$SUM0($5),EXPR$33=MAX($5),EXPR$34=MIN($5)), rowcount=100.0, cumulative cost={inf}
          		rel#24:LogicalProject.NONE.[](input=rel#23:Subset#4.NONE.[],$f0=CASE(=($1, 0), null, $0),$f1=CASE(=($3, 0), null, $2),$f2=$3,EXPR$2=$2,EXPR$3=$4,EXPR$4=$5,$f6=CASE(=($7, 0), null, $6),$f7=CASE(=($9, 0), null, $8),$f8=$9,EXPR$8=$8,EXPR$9=$10,EXPR$10=$11,$f12=CASE(=($13, 0), null, $12),$f13=CASE(=($15, 0), null, $14),$f14=$15,EXPR$14=$14,EXPR$15=$16,EXPR$16=$17,$f18=CASE(=($19, 0), null, $18),$f19=CASE(=($21, 0), null, $20),$f20=$21,EXPR$20=$20,EXPR$21=$22,EXPR$22=$23,$f24=CASE(=($25, 0), null, $24),$f25=CASE(=($27, 0), null, $26),$f26=$27,EXPR$26=$26,EXPR$27=$28,EXPR$28=$29,$f30=CASE(=($31, 0), null, $30),$f31=CASE(=($33, 0), null, $32),$f32=$33,EXPR$32=$32,EXPR$33=$34,EXPR$34=$35), rowcount=100.0, cumulative cost={inf}
          		rel#30:LogicalCalc.NONE.[[]](input=rel#23:Subset#4.NONE.[],expr#0..35={inputs},expr#36=0,expr#37==($t1, $t36),expr#38=null,expr#39=CASE($t37, $t38, $t0),expr#40==($t3, $t36),expr#41=CASE($t40, $t38, $t2),expr#42==($t7, $t36),expr#43=CASE($t42, $t38, $t6),expr#44==($t9, $t36),expr#45=CASE($t44, $t38, $t8),expr#46==($t13, $t36),expr#47=CASE($t46, $t38, $t12),expr#48==($t15, $t36),expr#49=CASE($t48, $t38, $t14),expr#50==($t19, $t36),expr#51=CASE($t50, $t38, $t18),expr#52==($t21, $t36),expr#53=CASE($t52, $t38, $t20),expr#54==($t25, $t36),expr#55=CASE($t54, $t38, $t24),expr#56==($t27, $t36),expr#57=CASE($t56, $t38, $t26),expr#58==($t31, $t36),expr#59=CASE($t58, $t38, $t30),expr#60==($t33, $t36),expr#61=CASE($t60, $t38, $t32),$f0=$t39,$f1=$t41,$f2=$t3,EXPR$2=$t2,EXPR$3=$t4,EXPR$4=$t5,$f6=$t43,$f7=$t45,$f8=$t9,EXPR$8=$t8,EXPR$9=$t10,EXPR$10=$t11,$f12=$t47,$f13=$t49,$f14=$t15,EXPR$14=$t14,EXPR$15=$t16,EXPR$16=$t17,$f18=$t51,$f19=$t53,$f20=$t21,EXPR$20=$t20,EXPR$21=$t22,EXPR$22=$t23,$f24=$t55,$f25=$t57,$f26=$t27,EXPR$26=$t26,EXPR$27=$t28,EXPR$28=$t29,$f30=$t59,$f31=$t61,$f32=$t33,EXPR$32=$t32,EXPR$33=$t34,EXPR$34=$t35), rowcount=100.0, cumulative cost={inf}
          	rel#17:Subset#3.DATASET.[], best=null, importance=0.9
          		rel#39:DataSetCalc.DATASET.[[]](input=rel#32:Subset#4.DATASET.[],select=CASE(=($f1, 0), null, $f0) AS $f0, CASE(=($f3, 0), null, $f2) AS $f1, $f3 AS $f2, $f2 AS $f3, EXPR$3, EXPR$4, CASE(=($f7, 0), null, $f6) AS $f6, CASE(=($f9, 0), null, $f8) AS $f7, $f9 AS $f8, $f8 AS $f9, EXPR$9, EXPR$10, CASE(=($f13, 0), null, $f12) AS $f12, CASE(=($f15, 0), null, $f14) AS $f13, $f15 AS $f14, $f14 AS $f15, EXPR$15, EXPR$16, CASE(=($f19, 0), null, $f18) AS $f18, CASE(=($f21, 0), null, $f20) AS $f19, $f21 AS $f20, $f20 AS $f21, EXPR$21, EXPR$22, CASE(=($f25, 0), null, $f24) AS $f24, CASE(=($f27, 0), null, $f26) AS $f25, $f27 AS $f26, $f26 AS $f27, EXPR$27, EXPR$28, CASE(=($f31, 0), null, $f30) AS $f30, CASE(=($f33, 0), null, $f32) AS $f31, $f33 AS $f32, $f32 AS $f33, EXPR$33, EXPR$34), rowcount=100.0, cumulative cost={inf}
          Set#4, type: RecordType(TINYINT $f0, BIGINT $f1, TINYINT $f2, BIGINT $f3, TINYINT EXPR$3, TINYINT EXPR$4, SMALLINT $f6, BIGINT $f7, SMALLINT $f8, BIGINT $f9, SMALLINT EXPR$9, SMALLINT EXPR$10, INTEGER $f12, BIGINT $f13, INTEGER $f14, BIGINT $f15, INTEGER EXPR$15, INTEGER EXPR$16, BIGINT $f18, BIGINT $f19, BIGINT $f20, BIGINT $f21, BIGINT EXPR$21, BIGINT EXPR$22, FLOAT $f24, BIGINT $f25, FLOAT $f26, BIGINT $f27, FLOAT EXPR$27, FLOAT EXPR$28, DOUBLE $f30, BIGINT $f31, DOUBLE $f32, BIGINT $f33, DOUBLE EXPR$33, DOUBLE EXPR$34)
          	rel#23:Subset#4.NONE.[], best=null, importance=0.7290000000000001
          		rel#21:LogicalAggregate.NONE.[](input=rel#12:Subset#2.NONE.[],group={},agg#0=$SUM0($6),agg#1=COUNT($6),agg#2=$SUM0($0),agg#3=COUNT($0),EXPR$3=MAX($0),EXPR$4=MIN($0),agg#6=$SUM0($7),agg#7=COUNT($7),agg#8=$SUM0($1),agg#9=COUNT($1),EXPR$9=MAX($1),EXPR$10=MIN($1),agg#12=$SUM0($8),agg#13=COUNT($8),agg#14=$SUM0($2),agg#15=COUNT($2),EXPR$15=MAX($2),EXPR$16=MIN($2),agg#18=$SUM0($9),agg#19=COUNT($9),agg#20=$SUM0($3),agg#21=COUNT($3),EXPR$21=MAX($3),EXPR$22=MIN($3),agg#24=$SUM0($10),agg#25=COUNT($10),agg#26=$SUM0($4),agg#27=COUNT($4),EXPR$27=MAX($4),EXPR$28=MIN($4),agg#30=$SUM0($11),agg#31=COUNT($11),agg#32=$SUM0($5),agg#33=COUNT($5),EXPR$33=MAX($5),EXPR$34=MIN($5)), rowcount=100.0, cumulative cost={inf}
          	rel#32:Subset#4.DATASET.[], best=null, importance=0.81
          
          	at org.apache.calcite.plan.volcano.RelSubset$CheapestPlanReplacer.visit(RelSubset.java:443)
          	at org.apache.calcite.plan.volcano.RelSubset.buildCheapestPlan(RelSubset.java:293)
          	at org.apache.calcite.plan.volcano.VolcanoPlanner.findBestExp(VolcanoPlanner.java:841)
          	at org.apache.calcite.tools.Programs$RuleSetProgram.run(Programs.java:334)
          	at org.apache.flink.api.table.BatchTableEnvironment.translate(BatchTableEnvironment.scala:253)
          

          I think what RelOptPlanner can't choose rule for which he needs to handle this query and can't apply any RelNode suitable for this sql query. For fix it need creating common rule which will contain rules for simple aggregate functions (sum, max, min, count) and new functions (stddev_pop, stddev_samp, etc)
          What do you think about it?

          Show
          anmu Anton Mushin added a comment - Sorry, I will try explain my problem. I have two tests test1 @Test def testStddevPopAggregate(): Unit = { val env = ExecutionEnvironment.getExecutionEnvironment val tEnv = TableEnvironment.getTableEnvironment(env, config) val ds = env.fromElements( (1: Byte , 1 : Short , 1, 1L, 1F, 1D), (2: Byte , 2 : Short , 2, 2L, 2F, 2D)).toTable(tEnv) tEnv.registerTable( "myTable" , ds) val columns = Array( "_1" , "_2" , "_3" , "_4" , "_5" , "_6" ) val sqlQuery = getSelectQuery( "STDDEV_POP(?)" )(columns, "myTable" ) //val sqlExpectedQuery = getSelectQuery( "SQRT((SUM(? * ?) - SUM(?) * SUM(?) / COUNT(?)) / COUNT(?))" )(columns, "myTable" ) val actualResult = tEnv.sql(sqlQuery).toDataSet[Row].collect() //val expectedResult = tEnv.sql(sqlExpectedQuery).toDataSet[Row].collect().toString.replaceAll( "Buffer\\(|\\)" , "") val expectedResult = "0,0,0,0,0.5,0.5" TestBaseUtils.compareOrderedResultAsText(actualResult.asJava, expectedResult) } test2 @Test def testStddevPopAggregateWithOtherAggreagte(): Unit = { val localconf = config localconf.setNullCheck( true ) val env = ExecutionEnvironment.getExecutionEnvironment val tEnv = TableEnvironment.getTableEnvironment(env, localconf) val ds = env.fromElements( (1: Byte , 1 : Short , 1, 1L, 1F, 1D), (2: Byte , 2 : Short , 2, 2L, 2F, 2D)).toTable(tEnv) tEnv.registerTable( "myTable" , ds) val columns = Array( "_1" , "_2" , "_3" , "_4" , "_5" , "_6" ) val sqlQuery = getSelectQuery( "STDDEV_POP(?), avg(?), sum(?), max(?), min(?), count(?)" )(columns, "myTable" ) val sqlExpectedQuery = getSelectQuery( "SQRT((SUM(? * ?) - SUM(?) * SUM(?) / COUNT(?)) / COUNT(?))" + "avg(?),sum(?),max(?),min(?),count(?)" )(columns, "myTable" ) val actualResult = tEnv.sql(sqlQuery).toDataSet[Row].collect() val expectedResult = tEnv.sql(sqlExpectedQuery).toDataSet[Row].collect().toString.replaceAll( "Buffer\\(|\\)" , "") //val expectedResult = "0.0,1,3,2,1,2,0.0,1,3,2,1,2,0.0,1,3,2,1,2,0.0,1,3,2,1,2,0.5,1.5,3.0,2.0,1.0,2,0.5,1.5,3.0,2.0,1.0,2" TestBaseUtils.compareOrderedResultAsText(actualResult.asJava, expectedResult) } Actual code for running tests. First test is passed and second isn't. I have Calcite ST: org.apache.calcite.plan.RelOptPlanner$CannotPlanException: Node [rel#7:Subset#1.DATASET.[]] could not be implemented; planner state: Root: rel#7:Subset#1.DATASET.[] Original rel: Sets: Set#0, type: RecordType(TINYINT _1, SMALLINT _2, INTEGER _3, BIGINT _4, FLOAT _5, DOUBLE _6) rel#4:Subset#0.NONE.[], best=null, importance=0.81 rel#0:LogicalTableScan.NONE.[](table=[_DataSetTable_0]), rowcount=1000.0, cumulative cost={inf} rel#26:Subset#0.DATASET.[], best=rel#25, importance=0.405 rel#25:DataSetScan.DATASET.[](table=[_DataSetTable_0]), rowcount=1000.0, cumulative cost={1000.0 rows, 1000.0 cpu, 0.0 io} rel#28:Subset#0.ENUMERABLE.[], best=rel#27, importance=0.405 rel#27:EnumerableTableScan.ENUMERABLE.[](table=[_DataSetTable_0]), rowcount=1000.0, cumulative cost={1000.0 rows, 1001.0 cpu, 0.0 io} Set#1, type: RecordType(TINYINT EXPR$0, TINYINT EXPR$1, TINYINT EXPR$2, TINYINT EXPR$3, TINYINT EXPR$4, BIGINT EXPR$5, SMALLINT EXPR$6, SMALLINT EXPR$7, SMALLINT EXPR$8, SMALLINT EXPR$9, SMALLINT EXPR$10, BIGINT EXPR$11, INTEGER EXPR$12, INTEGER EXPR$13, INTEGER EXPR$14, INTEGER EXPR$15, INTEGER EXPR$16, BIGINT EXPR$17, BIGINT EXPR$18, BIGINT EXPR$19, BIGINT EXPR$20, BIGINT EXPR$21, BIGINT EXPR$22, BIGINT EXPR$23, FLOAT EXPR$24, FLOAT EXPR$25, FLOAT EXPR$26, FLOAT EXPR$27, FLOAT EXPR$28, BIGINT EXPR$29, DOUBLE EXPR$30, DOUBLE EXPR$31, DOUBLE EXPR$32, DOUBLE EXPR$33, DOUBLE EXPR$34, BIGINT EXPR$35) rel#6:Subset#1.NONE.[], best=null, importance=0.9 rel#5:LogicalAggregate.NONE.[](input=rel#4:Subset#0.NONE.[],group={},EXPR$0=STDDEV_POP($0),EXPR$1=AVG($0),EXPR$2=SUM($0),EXPR$3=MAX($0),EXPR$4=MIN($0),EXPR$5=COUNT($0),EXPR$6=STDDEV_POP($1),EXPR$7=AVG($1),EXPR$8=SUM($1),EXPR$9=MAX($1),EXPR$10=MIN($1),EXPR$11=COUNT($1),EXPR$12=STDDEV_POP($2),EXPR$13=AVG($2),EXPR$14=SUM($2),EXPR$15=MAX($2),EXPR$16=MIN($2),EXPR$17=COUNT($2),EXPR$18=STDDEV_POP($3),EXPR$19=AVG($3),EXPR$20=SUM($3),EXPR$21=MAX($3),EXPR$22=MIN($3),EXPR$23=COUNT($3),EXPR$24=STDDEV_POP($4),EXPR$25=AVG($4),EXPR$26=SUM($4),EXPR$27=MAX($4),EXPR$28=MIN($4),EXPR$29=COUNT($4),EXPR$30=STDDEV_POP($5),EXPR$31=AVG($5),EXPR$32=SUM($5),EXPR$33=MAX($5),EXPR$34=MIN($5),EXPR$35=COUNT($5)), rowcount=100.0, cumulative cost={inf} rel#15:LogicalProject.NONE.[](input=rel#14:Subset#3.NONE.[],EXPR$0=CAST(POWER(/(-($0, /(*($1, $1), $2)), $2), 0.5)):TINYINT,EXPR$1=CAST(/($1, $2)):TINYINT,EXPR$2=CASE(=($2, 0), null, $3),EXPR$3=$4,EXPR$4=$5,EXPR$5=$2,EXPR$6=CAST(POWER(/(-($6, /(*($7, $7), $8)), $8), 0.5)):SMALLINT,EXPR$7=CAST(/($7, $8)):SMALLINT,EXPR$8=CASE(=($8, 0), null, $9),EXPR$9=$10,EXPR$10=$11,EXPR$11=$8,EXPR$12=CAST(POWER(/(-($12, /(*($13, $13), $14)), $14), 0.5)):INTEGER,EXPR$13=CAST(/($13, $14)):INTEGER,EXPR$14=CASE(=($14, 0), null, $15),EXPR$15=$16,EXPR$16=$17,EXPR$17=$14,EXPR$18=CAST(POWER(/(-($18, /(*($19, $19), $20)), $20), 0.5)):BIGINT,EXPR$19=CAST(/($19, $20)):BIGINT,EXPR$20=CASE(=($20, 0), null, $21),EXPR$21=$22,EXPR$22=$23,EXPR$23=$20,EXPR$24=CAST(POWER(/(-($24, /(*($25, $25), $26)), $26), 0.5)):FLOAT,EXPR$25=CAST(/($25, $26)):FLOAT,EXPR$26=CASE(=($26, 0), null, $27),EXPR$27=$28,EXPR$28=$29,EXPR$29=$26,EXPR$30=CAST(POWER(/(-($30, /(*($31, $31), $32)), $32), 0.5)):DOUBLE,EXPR$31=CAST(/($31, $32)):DOUBLE,EXPR$32=CASE(=($32, 0), null, $33),EXPR$33=$34,EXPR$34=$35,EXPR$35=$32), rowcount=100.0, cumulative cost={inf} rel#16:LogicalCalc.NONE.[[]](input=rel#14:Subset#3.NONE.[],expr#0..35={inputs},expr#36=*($t1, $t1),expr#37=/($t36, $t2),expr#38=-($t0, $t37),expr#39=/($t38, $t2),expr#40=0.5,expr#41=POWER($t39, $t40),expr#42=CAST($t41):TINYINT,expr#43=/($t1, $t2),expr#44=CAST($t43):TINYINT,expr#45=0,expr#46==($t2, $t45),expr#47=null,expr#48=CASE($t46, $t47, $t3),expr#49=*($t7, $t7),expr#50=/($t49, $t8),expr#51=-($t6, $t50),expr#52=/($t51, $t8),expr#53=POWER($t52, $t40),expr#54=CAST($t53):SMALLINT,expr#55=/($t7, $t8),expr#56=CAST($t55):SMALLINT,expr#57==($t8, $t45),expr#58=CASE($t57, $t47, $t9),expr#59=*($t13, $t13),expr#60=/($t59, $t14),expr#61=-($t12, $t60),expr#62=/($t61, $t14),expr#63=POWER($t62, $t40),expr#64=CAST($t63):INTEGER,expr#65=/($t13, $t14),expr#66=CAST($t65):INTEGER,expr#67==($t14, $t45),expr#68=CASE($t67, $t47, $t15),expr#69=*($t19, $t19),expr#70=/($t69, $t20),expr#71=-($t18, $t70),expr#72=/($t71, $t20),expr#73=POWER($t72, $t40),expr#74=CAST($t73):BIGINT,expr#75=/($t19, $t20),expr#76=CAST($t75):BIGINT,expr#77==($t20, $t45),expr#78=CASE($t77, $t47, $t21),expr#79=*($t25, $t25),expr#80=/($t79, $t26),expr#81=-($t24, $t80),expr#82=/($t81, $t26),expr#83=POWER($t82, $t40),expr#84=CAST($t83):FLOAT,expr#85=/($t25, $t26),expr#86=CAST($t85):FLOAT,expr#87==($t26, $t45),expr#88=CASE($t87, $t47, $t27),expr#89=*($t31, $t31),expr#90=/($t89, $t32),expr#91=-($t30, $t90),expr#92=/($t91, $t32),expr#93=POWER($t92, $t40),expr#94=CAST($t93):DOUBLE,expr#95=/($t31, $t32),expr#96=CAST($t95):DOUBLE,expr#97==($t32, $t45),expr#98=CASE($t97, $t47, $t33),EXPR$0=$t42,EXPR$1=$t44,EXPR$2=$t48,EXPR$3=$t4,EXPR$4=$t5,EXPR$5=$t2,EXPR$6=$t54,EXPR$7=$t56,EXPR$8=$t58,EXPR$9=$t10,EXPR$10=$t11,EXPR$11=$t8,EXPR$12=$t64,EXPR$13=$t66,EXPR$14=$t68,EXPR$15=$t16,EXPR$16=$t17,EXPR$17=$t14,EXPR$18=$t74,EXPR$19=$t76,EXPR$20=$t78,EXPR$21=$t22,EXPR$22=$t23,EXPR$23=$t20,EXPR$24=$t84,EXPR$25=$t86,EXPR$26=$t88,EXPR$27=$t28,EXPR$28=$t29,EXPR$29=$t26,EXPR$30=$t94,EXPR$31=$t96,EXPR$32=$t98,EXPR$33=$t34,EXPR$34=$t35,EXPR$35=$t32), importance=0.0, rowcount=100.0, cumulative cost={inf} rel#19:LogicalCalc.NONE.[[]](input=rel#14:Subset#3.NONE.[],expr#0..35={inputs},expr#36=*($t1, $t1),expr#37=/($t36, $t2),expr#38=-($t0, $t37),expr#39=/($t38, $t2),expr#40=0.5,expr#41=POWER($t39, $t40),expr#42=CAST($t41):TINYINT,expr#43=/($t1, $t2),expr#44=CAST($t43):TINYINT,expr#45=0,expr#46==($t2, $t45),expr#47=null,expr#48=CASE($t46, $t47, $t3),expr#49=*($t7, $t7),expr#50=/($t49, $t8),expr#51=-($t6, $t50),expr#52=/($t51, $t8),expr#53=POWER($t52, $t40),expr#54=CAST($t53):SMALLINT,expr#55=/($t7, $t8),expr#56=CAST($t55):SMALLINT,expr#57==($t8, $t45),expr#58=CASE($t57, $t47, $t9),expr#59=*($t13, $t13),expr#60=/($t59, $t14),expr#61=-($t12, $t60),expr#62=/($t61, $t14),expr#63=POWER($t62, $t40),expr#64=CAST($t63):INTEGER,expr#65=/($t13, $t14),expr#66=CAST($t65):INTEGER,expr#67==($t14, $t45),expr#68=CASE($t67, $t47, $t15),expr#69=*($t19, $t19),expr#70=/($t69, $t20),expr#71=-($t18, $t70),expr#72=/($t71, $t20),expr#73=POWER($t72, $t40),expr#74=CAST($t73):BIGINT,expr#75=/($t19, $t20),expr#76==($t20, $t45),expr#77=CASE($t76, $t47, $t21),expr#78=*($t25, $t25),expr#79=/($t78, $t26),expr#80=-($t24, $t79),expr#81=/($t80, $t26),expr#82=POWER($t81, $t40),expr#83=CAST($t82):FLOAT,expr#84=/($t25, $t26),expr#85==($t26, $t45),expr#86=CASE($t85, $t47, $t27),expr#87=*($t31, $t31),expr#88=/($t87, $t32),expr#89=-($t30, $t88),expr#90=/($t89, $t32),expr#91=POWER($t90, $t40),expr#92=/($t31, $t32),expr#93==($t32, $t45),expr#94=CASE($t93, $t47, $t33),EXPR$0=$t42,EXPR$1=$t44,EXPR$2=$t48,EXPR$3=$t4,EXPR$4=$t5,EXPR$5=$t2,EXPR$6=$t54,EXPR$7=$t56,EXPR$8=$t58,EXPR$9=$t10,EXPR$10=$t11,EXPR$11=$t8,EXPR$12=$t64,EXPR$13=$t66,EXPR$14=$t68,EXPR$15=$t16,EXPR$16=$t17,EXPR$17=$t14,EXPR$18=$t74,EXPR$19=$t75,EXPR$20=$t77,EXPR$21=$t22,EXPR$22=$t23,EXPR$23=$t20,EXPR$24=$t83,EXPR$25=$t84,EXPR$26=$t86,EXPR$27=$t28,EXPR$28=$t29,EXPR$29=$t26,EXPR$30=$t91,EXPR$31=$t92,EXPR$32=$t94,EXPR$33=$t34,EXPR$34=$t35,EXPR$35=$t32), rowcount=100.0, cumulative cost={inf} rel#31:LogicalCalc.NONE.[[]](input=rel#23:Subset#4.NONE.[],expr#0..35={inputs},expr#36=0,expr#37==($t1, $t36),expr#38=null,expr#39=CASE($t37, $t38, $t0),expr#40==($t3, $t36),expr#41=CASE($t40, $t38, $t2),expr#42=*($t41, $t41),expr#43=/($t42, $t3),expr#44=-($t39, $t43),expr#45=/($t44, $t3),expr#46=0.5,expr#47=POWER($t45, $t46),expr#48=CAST($t47):TINYINT,expr#49=/($t41, $t3),expr#50=CAST($t49):TINYINT,expr#51==($t7, $t36),expr#52=CASE($t51, $t38, $t6),expr#53==($t9, $t36),expr#54=CASE($t53, $t38, $t8),expr#55=*($t54, $t54),expr#56=/($t55, $t9),expr#57=-($t52, $t56),expr#58=/($t57, $t9),expr#59=POWER($t58, $t46),expr#60=CAST($t59):SMALLINT,expr#61=/($t54, $t9),expr#62=CAST($t61):SMALLINT,expr#63==($t13, $t36),expr#64=CASE($t63, $t38, $t12),expr#65==($t15, $t36),expr#66=CASE($t65, $t38, $t14),expr#67=*($t66, $t66),expr#68=/($t67, $t15),expr#69=-($t64, $t68),expr#70=/($t69, $t15),expr#71=POWER($t70, $t46),expr#72=CAST($t71):INTEGER,expr#73=/($t66, $t15),expr#74=CAST($t73):INTEGER,expr#75==($t19, $t36),expr#76=CASE($t75, $t38, $t18),expr#77==($t21, $t36),expr#78=CASE($t77, $t38, $t20),expr#79=*($t78, $t78),expr#80=/($t79, $t21),expr#81=-($t76, $t80),expr#82=/($t81, $t21),expr#83=POWER($t82, $t46),expr#84=CAST($t83):BIGINT,expr#85=/($t78, $t21),expr#86=CAST($t85):BIGINT,expr#87==($t25, $t36),expr#88=CASE($t87, $t38, $t24),expr#89==($t27, $t36),expr#90=CASE($t89, $t38, $t26),expr#91=*($t90, $t90),expr#92=/($t91, $t27),expr#93=-($t88, $t92),expr#94=/($t93, $t27),expr#95=POWER($t94, $t46),expr#96=CAST($t95):FLOAT,expr#97=/($t90, $t27),expr#98=CAST($t97):FLOAT,expr#99==($t31, $t36),expr#100=CASE($t99, $t38, $t30),expr#101==($t33, $t36),expr#102=CASE($t101, $t38, $t32),expr#103=*($t102, $t102),expr#104=/($t103, $t33),expr#105=-($t100, $t104),expr#106=/($t105, $t33),expr#107=POWER($t106, $t46),expr#108=CAST($t107):DOUBLE,expr#109=/($t102, $t33),expr#110=CAST($t109):DOUBLE,EXPR$0=$t48,EXPR$1=$t50,EXPR$2=$t41,EXPR$3=$t4,EXPR$4=$t5,EXPR$5=$t3,EXPR$6=$t60,EXPR$7=$t62,EXPR$8=$t54,EXPR$9=$t10,EXPR$10=$t11,EXPR$11=$t9,EXPR$12=$t72,EXPR$13=$t74,EXPR$14=$t66,EXPR$15=$t16,EXPR$16=$t17,EXPR$17=$t15,EXPR$18=$t84,EXPR$19=$t86,EXPR$20=$t78,EXPR$21=$t22,EXPR$22=$t23,EXPR$23=$t21,EXPR$24=$t96,EXPR$25=$t98,EXPR$26=$t90,EXPR$27=$t28,EXPR$28=$t29,EXPR$29=$t27,EXPR$30=$t108,EXPR$31=$t110,EXPR$32=$t102,EXPR$33=$t34,EXPR$34=$t35,EXPR$35=$t33), importance=0.0, rowcount=100.0, cumulative cost={inf} rel#34:LogicalCalc.NONE.[[]](input=rel#23:Subset#4.NONE.[],expr#0..35={inputs},expr#36=0,expr#37==($t1, $t36),expr#38=null,expr#39=CASE($t37, $t38, $t0),expr#40==($t3, $t36),expr#41=CASE($t40, $t38, $t2),expr#42=*($t41, $t41),expr#43=/($t42, $t3),expr#44=-($t39, $t43),expr#45=/($t44, $t3),expr#46=0.5,expr#47=POWER($t45, $t46),expr#48=CAST($t47):TINYINT,expr#49=/($t41, $t3),expr#50=CAST($t49):TINYINT,expr#51==($t7, $t36),expr#52=CASE($t51, $t38, $t6),expr#53==($t9, $t36),expr#54=CASE($t53, $t38, $t8),expr#55=*($t54, $t54),expr#56=/($t55, $t9),expr#57=-($t52, $t56),expr#58=/($t57, $t9),expr#59=POWER($t58, $t46),expr#60=CAST($t59):SMALLINT,expr#61=/($t54, $t9),expr#62=CAST($t61):SMALLINT,expr#63==($t13, $t36),expr#64=CASE($t63, $t38, $t12),expr#65==($t15, $t36),expr#66=CASE($t65, $t38, $t14),expr#67=*($t66, $t66),expr#68=/($t67, $t15),expr#69=-($t64, $t68),expr#70=/($t69, $t15),expr#71=POWER($t70, $t46),expr#72=CAST($t71):INTEGER,expr#73=/($t66, $t15),expr#74=CAST($t73):INTEGER,expr#75==($t19, $t36),expr#76=CASE($t75, $t38, $t18),expr#77==($t21, $t36),expr#78=CASE($t77, $t38, $t20),expr#79=*($t78, $t78),expr#80=/($t79, $t21),expr#81=-($t76, $t80),expr#82=/($t81, $t21),expr#83=POWER($t82, $t46),expr#84=CAST($t83):BIGINT,expr#85=/($t78, $t21),expr#86==($t25, $t36),expr#87=CASE($t86, $t38, $t24),expr#88==($t27, $t36),expr#89=CASE($t88, $t38, $t26),expr#90=*($t89, $t89),expr#91=/($t90, $t27),expr#92=-($t87, $t91),expr#93=/($t92, $t27),expr#94=POWER($t93, $t46),expr#95=CAST($t94):FLOAT,expr#96=/($t89, $t27),expr#97==($t31, $t36),expr#98=CASE($t97, $t38, $t30),expr#99==($t33, $t36),expr#100=CASE($t99, $t38, $t32),expr#101=*($t100, $t100),expr#102=/($t101, $t33),expr#103=-($t98, $t102),expr#104=/($t103, $t33),expr#105=POWER($t104, $t46),expr#106=/($t100, $t33),EXPR$0=$t48,EXPR$1=$t50,EXPR$2=$t41,EXPR$3=$t4,EXPR$4=$t5,EXPR$5=$t3,EXPR$6=$t60,EXPR$7=$t62,EXPR$8=$t54,EXPR$9=$t10,EXPR$10=$t11,EXPR$11=$t9,EXPR$12=$t72,EXPR$13=$t74,EXPR$14=$t66,EXPR$15=$t16,EXPR$16=$t17,EXPR$17=$t15,EXPR$18=$t84,EXPR$19=$t85,EXPR$20=$t78,EXPR$21=$t22,EXPR$22=$t23,EXPR$23=$t21,EXPR$24=$t95,EXPR$25=$t96,EXPR$26=$t89,EXPR$27=$t28,EXPR$28=$t29,EXPR$29=$t27,EXPR$30=$t105,EXPR$31=$t106,EXPR$32=$t100,EXPR$33=$t34,EXPR$34=$t35,EXPR$35=$t33), rowcount=100.0, cumulative cost={inf} rel#7:Subset#1.DATASET.[], best=null, importance=1.0 rel#8:AbstractConverter.DATASET.[](input=rel#6:Subset#1.NONE.[],convention=DATASET,sort=[]), rowcount=100.0, cumulative cost={inf} rel#18:DataSetCalc.DATASET.[[]](input=rel#17:Subset#3.DATASET.[],select=CAST(POWER(/(-($f0, /(*($f1, $f1), $f2)), $f2), 0.5)) AS $f0, CAST(/($f1, $f2)) AS $f1, CASE(=($f2, 0), null, EXPR$2) AS $f2, EXPR$3 AS EXPR$2, EXPR$4 AS EXPR$3, $f2 AS EXPR$4, CAST(POWER(/(-($f6, /(*($f7, $f7), $f8)), $f8), 0.5)) AS $f6, CAST(/($f7, $f8)) AS $f7, CASE(=($f8, 0), null, EXPR$8) AS $f8, EXPR$9 AS EXPR$8, EXPR$10 AS EXPR$9, $f8 AS EXPR$10, CAST(POWER(/(-($f12, /(*($f13, $f13), $f14)), $f14), 0.5)) AS $f12, CAST(/($f13, $f14)) AS $f13, CASE(=($f14, 0), null, EXPR$14) AS $f14, EXPR$15 AS EXPR$14, EXPR$16 AS EXPR$15, $f14 AS EXPR$16, CAST(POWER(/(-($f18, /(*($f19, $f19), $f20)), $f20), 0.5)) AS $f18, CAST(/($f19, $f20)) AS $f19, CASE(=($f20, 0), null, EXPR$20) AS $f20, EXPR$21 AS EXPR$20, EXPR$22 AS EXPR$21, $f20 AS EXPR$22, CAST(POWER(/(-($f24, /(*($f25, $f25), $f26)), $f26), 0.5)) AS $f24, CAST(/($f25, $f26)) AS $f25, CASE(=($f26, 0), null, EXPR$26) AS $f26, EXPR$27 AS EXPR$26, EXPR$28 AS EXPR$27, $f26 AS EXPR$28, CAST(POWER(/(-($f30, /(*($f31, $f31), $f32)), $f32), 0.5)) AS $f30, CAST(/($f31, $f32)) AS $f31, CASE(=($f32, 0), null, EXPR$32) AS $f32, EXPR$33 AS EXPR$32, EXPR$34 AS EXPR$33, $f32 AS EXPR$34), rowcount=100.0, cumulative cost={inf} rel#20:DataSetCalc.DATASET.[[]](input=rel#17:Subset#3.DATASET.[],select=CAST(POWER(/(-($f0, /(*($f1, $f1), $f2)), $f2), 0.5)) AS $f0, CAST(/($f1, $f2)) AS $f1, CASE(=($f2, 0), null, EXPR$2) AS $f2, EXPR$3 AS EXPR$2, EXPR$4 AS EXPR$3, $f2 AS EXPR$4, CAST(POWER(/(-($f6, /(*($f7, $f7), $f8)), $f8), 0.5)) AS $f6, CAST(/($f7, $f8)) AS $f7, CASE(=($f8, 0), null, EXPR$8) AS $f8, EXPR$9 AS EXPR$8, EXPR$10 AS EXPR$9, $f8 AS EXPR$10, CAST(POWER(/(-($f12, /(*($f13, $f13), $f14)), $f14), 0.5)) AS $f12, CAST(/($f13, $f14)) AS $f13, CASE(=($f14, 0), null, EXPR$14) AS $f14, EXPR$15 AS EXPR$14, EXPR$16 AS EXPR$15, $f14 AS EXPR$16, CAST(POWER(/(-($f18, /(*($f19, $f19), $f20)), $f20), 0.5)) AS $f18, /($f19, $f20) AS $f19, CASE(=($f20, 0), null, EXPR$20) AS $f20, EXPR$21 AS EXPR$20, EXPR$22 AS EXPR$21, $f20 AS EXPR$22, CAST(POWER(/(-($f24, /(*($f25, $f25), $f26)), $f26), 0.5)) AS $f24, /($f25, $f26) AS $f25, CASE(=($f26, 0), null, EXPR$26) AS $f26, EXPR$27 AS EXPR$26, EXPR$28 AS EXPR$27, $f26 AS EXPR$28, POWER(/(-($f30, /(*($f31, $f31), $f32)), $f32), 0.5) AS $f30, /($f31, $f32) AS $f31, CASE(=($f32, 0), null, EXPR$32) AS $f32, EXPR$33 AS EXPR$32, EXPR$34 AS EXPR$33, $f32 AS EXPR$34), rowcount=100.0, cumulative cost={inf} rel#33:DataSetCalc.DATASET.[[]](input=rel#32:Subset#4.DATASET.[],select=CAST(POWER(/(-(CASE(=($f1, 0), null, $f0), /(*(CASE(=($f3, 0), null, $f2), CASE(=($f3, 0), null, $f2)), $f3)), $f3), 0.5)) AS $f0, CAST(/(CASE(=($f3, 0), null, $f2), $f3)) AS $f1, CASE(=($f3, 0), null, $f2) AS $f2, EXPR$3 AS $f3, EXPR$4 AS EXPR$3, $f3 AS EXPR$4, CAST(POWER(/(-(CASE(=($f7, 0), null, $f6), /(*(CASE(=($f9, 0), null, $f8), CASE(=($f9, 0), null, $f8)), $f9)), $f9), 0.5)) AS $f6, CAST(/(CASE(=($f9, 0), null, $f8), $f9)) AS $f7, CASE(=($f9, 0), null, $f8) AS $f8, EXPR$9 AS $f9, EXPR$10 AS EXPR$9, $f9 AS EXPR$10, CAST(POWER(/(-(CASE(=($f13, 0), null, $f12), /(*(CASE(=($f15, 0), null, $f14), CASE(=($f15, 0), null, $f14)), $f15)), $f15), 0.5)) AS $f12, CAST(/(CASE(=($f15, 0), null, $f14), $f15)) AS $f13, CASE(=($f15, 0), null, $f14) AS $f14, EXPR$15 AS $f15, EXPR$16 AS EXPR$15, $f15 AS EXPR$16, CAST(POWER(/(-(CASE(=($f19, 0), null, $f18), /(*(CASE(=($f21, 0), null, $f20), CASE(=($f21, 0), null, $f20)), $f21)), $f21), 0.5)) AS $f18, CAST(/(CASE(=($f21, 0), null, $f20), $f21)) AS $f19, CASE(=($f21, 0), null, $f20) AS $f20, EXPR$21 AS $f21, EXPR$22 AS EXPR$21, $f21 AS EXPR$22, CAST(POWER(/(-(CASE(=($f25, 0), null, $f24), /(*(CASE(=($f27, 0), null, $f26), CASE(=($f27, 0), null, $f26)), $f27)), $f27), 0.5)) AS $f24, CAST(/(CASE(=($f27, 0), null, $f26), $f27)) AS $f25, CASE(=($f27, 0), null, $f26) AS $f26, EXPR$27 AS $f27, EXPR$28 AS EXPR$27, $f27 AS EXPR$28, CAST(POWER(/(-(CASE(=($f31, 0), null, $f30), /(*(CASE(=($f33, 0), null, $f32), CASE(=($f33, 0), null, $f32)), $f33)), $f33), 0.5)) AS $f30, CAST(/(CASE(=($f33, 0), null, $f32), $f33)) AS $f31, CASE(=($f33, 0), null, $f32) AS $f32, EXPR$33 AS $f33, EXPR$34 AS EXPR$33, $f33 AS EXPR$34), rowcount=100.0, cumulative cost={inf} rel#35:DataSetCalc.DATASET.[[]](input=rel#32:Subset#4.DATASET.[],select=CAST(POWER(/(-(CASE(=($f1, 0), null, $f0), /(*(CASE(=($f3, 0), null, $f2), CASE(=($f3, 0), null, $f2)), $f3)), $f3), 0.5)) AS $f0, CAST(/(CASE(=($f3, 0), null, $f2), $f3)) AS $f1, CASE(=($f3, 0), null, $f2) AS $f2, EXPR$3 AS $f3, EXPR$4 AS EXPR$3, $f3 AS EXPR$4, CAST(POWER(/(-(CASE(=($f7, 0), null, $f6), /(*(CASE(=($f9, 0), null, $f8), CASE(=($f9, 0), null, $f8)), $f9)), $f9), 0.5)) AS $f6, CAST(/(CASE(=($f9, 0), null, $f8), $f9)) AS $f7, CASE(=($f9, 0), null, $f8) AS $f8, EXPR$9 AS $f9, EXPR$10 AS EXPR$9, $f9 AS EXPR$10, CAST(POWER(/(-(CASE(=($f13, 0), null, $f12), /(*(CASE(=($f15, 0), null, $f14), CASE(=($f15, 0), null, $f14)), $f15)), $f15), 0.5)) AS $f12, CAST(/(CASE(=($f15, 0), null, $f14), $f15)) AS $f13, CASE(=($f15, 0), null, $f14) AS $f14, EXPR$15 AS $f15, EXPR$16 AS EXPR$15, $f15 AS EXPR$16, CAST(POWER(/(-(CASE(=($f19, 0), null, $f18), /(*(CASE(=($f21, 0), null, $f20), CASE(=($f21, 0), null, $f20)), $f21)), $f21), 0.5)) AS $f18, /(CASE(=($f21, 0), null, $f20), $f21) AS $f19, CASE(=($f21, 0), null, $f20) AS $f20, EXPR$21 AS $f21, EXPR$22 AS EXPR$21, $f21 AS EXPR$22, CAST(POWER(/(-(CASE(=($f25, 0), null, $f24), /(*(CASE(=($f27, 0), null, $f26), CASE(=($f27, 0), null, $f26)), $f27)), $f27), 0.5)) AS $f24, /(CASE(=($f27, 0), null, $f26), $f27) AS $f25, CASE(=($f27, 0), null, $f26) AS $f26, EXPR$27 AS $f27, EXPR$28 AS EXPR$27, $f27 AS EXPR$28, POWER(/(-(CASE(=($f31, 0), null, $f30), /(*(CASE(=($f33, 0), null, $f32), CASE(=($f33, 0), null, $f32)), $f33)), $f33), 0.5) AS $f30, /(CASE(=($f33, 0), null, $f32), $f33) AS $f31, CASE(=($f33, 0), null, $f32) AS $f32, EXPR$33 AS $f33, EXPR$34 AS EXPR$33, $f33 AS EXPR$34), rowcount=100.0, cumulative cost={inf} Set#2, type: RecordType(TINYINT _1, SMALLINT _2, INTEGER _3, BIGINT _4, FLOAT _5, DOUBLE _6, TINYINT $f6, SMALLINT $f7, INTEGER $f8, BIGINT $f9, FLOAT $f10, DOUBLE $f11) rel#12:Subset#2.NONE.[], best=null, importance=0.7290000000000001 rel#9:LogicalProject.NONE.[](input=rel#4:Subset#0.NONE.[],_1=$0,_2=$1,_3=$2,_4=$3,_5=$4,_6=$5,$f6=*($0, $0),$f7=*($1, $1),$f8=*($2, $2),$f9=*($3, $3),$f10=*($4, $4),$f11=*($5, $5)), rowcount=1000.0, cumulative cost={inf} rel#29:LogicalCalc.NONE.[[]](input=rel#4:Subset#0.NONE.[],expr#0..5={inputs},expr#6=*($t0, $t0),expr#7=*($t1, $t1),expr#8=*($t2, $t2),expr#9=*($t3, $t3),expr#10=*($t4, $t4),expr#11=*($t5, $t5),_1=$t0,_2=$t1,_3=$t2,_4=$t3,_5=$t4,_6=$t5,$f6=$t6,$f7=$t7,$f8=$t8,$f9=$t9,$f10=$t10,$f11=$t11), rowcount=1000.0, cumulative cost={inf} rel#38:Subset#2.DATASET.[], best=rel#37, importance=0.36450000000000005 rel#37:DataSetCalc.DATASET.[[]](input=rel#26:Subset#0.DATASET.[],select=_1, _2, _3, _4, _5, _6), rowcount=1000.0, cumulative cost={2000.0 rows, 25000.0 cpu, 0.0 io} Set#3, type: RecordType(TINYINT $f0, TINYINT $f1, BIGINT $f2, TINYINT EXPR$2, TINYINT EXPR$3, TINYINT EXPR$4, SMALLINT $f6, SMALLINT $f7, BIGINT $f8, SMALLINT EXPR$8, SMALLINT EXPR$9, SMALLINT EXPR$10, INTEGER $f12, INTEGER $f13, BIGINT $f14, INTEGER EXPR$14, INTEGER EXPR$15, INTEGER EXPR$16, BIGINT $f18, BIGINT $f19, BIGINT $f20, BIGINT EXPR$20, BIGINT EXPR$21, BIGINT EXPR$22, FLOAT $f24, FLOAT $f25, BIGINT $f26, FLOAT EXPR$26, FLOAT EXPR$27, FLOAT EXPR$28, DOUBLE $f30, DOUBLE $f31, BIGINT $f32, DOUBLE EXPR$32, DOUBLE EXPR$33, DOUBLE EXPR$34) rel#14:Subset#3.NONE.[], best=null, importance=0.81 rel#13:LogicalAggregate.NONE.[](input=rel#12:Subset#2.NONE.[],group={},agg#0=SUM($6),agg#1=SUM($0),agg#2=COUNT($0),EXPR$2=$SUM0($0),EXPR$3=MAX($0),EXPR$4=MIN($0),agg#6=SUM($7),agg#7=SUM($1),agg#8=COUNT($1),EXPR$8=$SUM0($1),EXPR$9=MAX($1),EXPR$10=MIN($1),agg#12=SUM($8),agg#13=SUM($2),agg#14=COUNT($2),EXPR$14=$SUM0($2),EXPR$15=MAX($2),EXPR$16=MIN($2),agg#18=SUM($9),agg#19=SUM($3),agg#20=COUNT($3),EXPR$20=$SUM0($3),EXPR$21=MAX($3),EXPR$22=MIN($3),agg#24=SUM($10),agg#25=SUM($4),agg#26=COUNT($4),EXPR$26=$SUM0($4),EXPR$27=MAX($4),EXPR$28=MIN($4),agg#30=SUM($11),agg#31=SUM($5),agg#32=COUNT($5),EXPR$32=$SUM0($5),EXPR$33=MAX($5),EXPR$34=MIN($5)), rowcount=100.0, cumulative cost={inf} rel#24:LogicalProject.NONE.[](input=rel#23:Subset#4.NONE.[],$f0=CASE(=($1, 0), null, $0),$f1=CASE(=($3, 0), null, $2),$f2=$3,EXPR$2=$2,EXPR$3=$4,EXPR$4=$5,$f6=CASE(=($7, 0), null, $6),$f7=CASE(=($9, 0), null, $8),$f8=$9,EXPR$8=$8,EXPR$9=$10,EXPR$10=$11,$f12=CASE(=($13, 0), null, $12),$f13=CASE(=($15, 0), null, $14),$f14=$15,EXPR$14=$14,EXPR$15=$16,EXPR$16=$17,$f18=CASE(=($19, 0), null, $18),$f19=CASE(=($21, 0), null, $20),$f20=$21,EXPR$20=$20,EXPR$21=$22,EXPR$22=$23,$f24=CASE(=($25, 0), null, $24),$f25=CASE(=($27, 0), null, $26),$f26=$27,EXPR$26=$26,EXPR$27=$28,EXPR$28=$29,$f30=CASE(=($31, 0), null, $30),$f31=CASE(=($33, 0), null, $32),$f32=$33,EXPR$32=$32,EXPR$33=$34,EXPR$34=$35), rowcount=100.0, cumulative cost={inf} rel#30:LogicalCalc.NONE.[[]](input=rel#23:Subset#4.NONE.[],expr#0..35={inputs},expr#36=0,expr#37==($t1, $t36),expr#38=null,expr#39=CASE($t37, $t38, $t0),expr#40==($t3, $t36),expr#41=CASE($t40, $t38, $t2),expr#42==($t7, $t36),expr#43=CASE($t42, $t38, $t6),expr#44==($t9, $t36),expr#45=CASE($t44, $t38, $t8),expr#46==($t13, $t36),expr#47=CASE($t46, $t38, $t12),expr#48==($t15, $t36),expr#49=CASE($t48, $t38, $t14),expr#50==($t19, $t36),expr#51=CASE($t50, $t38, $t18),expr#52==($t21, $t36),expr#53=CASE($t52, $t38, $t20),expr#54==($t25, $t36),expr#55=CASE($t54, $t38, $t24),expr#56==($t27, $t36),expr#57=CASE($t56, $t38, $t26),expr#58==($t31, $t36),expr#59=CASE($t58, $t38, $t30),expr#60==($t33, $t36),expr#61=CASE($t60, $t38, $t32),$f0=$t39,$f1=$t41,$f2=$t3,EXPR$2=$t2,EXPR$3=$t4,EXPR$4=$t5,$f6=$t43,$f7=$t45,$f8=$t9,EXPR$8=$t8,EXPR$9=$t10,EXPR$10=$t11,$f12=$t47,$f13=$t49,$f14=$t15,EXPR$14=$t14,EXPR$15=$t16,EXPR$16=$t17,$f18=$t51,$f19=$t53,$f20=$t21,EXPR$20=$t20,EXPR$21=$t22,EXPR$22=$t23,$f24=$t55,$f25=$t57,$f26=$t27,EXPR$26=$t26,EXPR$27=$t28,EXPR$28=$t29,$f30=$t59,$f31=$t61,$f32=$t33,EXPR$32=$t32,EXPR$33=$t34,EXPR$34=$t35), rowcount=100.0, cumulative cost={inf} rel#17:Subset#3.DATASET.[], best=null, importance=0.9 rel#39:DataSetCalc.DATASET.[[]](input=rel#32:Subset#4.DATASET.[],select=CASE(=($f1, 0), null, $f0) AS $f0, CASE(=($f3, 0), null, $f2) AS $f1, $f3 AS $f2, $f2 AS $f3, EXPR$3, EXPR$4, CASE(=($f7, 0), null, $f6) AS $f6, CASE(=($f9, 0), null, $f8) AS $f7, $f9 AS $f8, $f8 AS $f9, EXPR$9, EXPR$10, CASE(=($f13, 0), null, $f12) AS $f12, CASE(=($f15, 0), null, $f14) AS $f13, $f15 AS $f14, $f14 AS $f15, EXPR$15, EXPR$16, CASE(=($f19, 0), null, $f18) AS $f18, CASE(=($f21, 0), null, $f20) AS $f19, $f21 AS $f20, $f20 AS $f21, EXPR$21, EXPR$22, CASE(=($f25, 0), null, $f24) AS $f24, CASE(=($f27, 0), null, $f26) AS $f25, $f27 AS $f26, $f26 AS $f27, EXPR$27, EXPR$28, CASE(=($f31, 0), null, $f30) AS $f30, CASE(=($f33, 0), null, $f32) AS $f31, $f33 AS $f32, $f32 AS $f33, EXPR$33, EXPR$34), rowcount=100.0, cumulative cost={inf} Set#4, type: RecordType(TINYINT $f0, BIGINT $f1, TINYINT $f2, BIGINT $f3, TINYINT EXPR$3, TINYINT EXPR$4, SMALLINT $f6, BIGINT $f7, SMALLINT $f8, BIGINT $f9, SMALLINT EXPR$9, SMALLINT EXPR$10, INTEGER $f12, BIGINT $f13, INTEGER $f14, BIGINT $f15, INTEGER EXPR$15, INTEGER EXPR$16, BIGINT $f18, BIGINT $f19, BIGINT $f20, BIGINT $f21, BIGINT EXPR$21, BIGINT EXPR$22, FLOAT $f24, BIGINT $f25, FLOAT $f26, BIGINT $f27, FLOAT EXPR$27, FLOAT EXPR$28, DOUBLE $f30, BIGINT $f31, DOUBLE $f32, BIGINT $f33, DOUBLE EXPR$33, DOUBLE EXPR$34) rel#23:Subset#4.NONE.[], best=null, importance=0.7290000000000001 rel#21:LogicalAggregate.NONE.[](input=rel#12:Subset#2.NONE.[],group={},agg#0=$SUM0($6),agg#1=COUNT($6),agg#2=$SUM0($0),agg#3=COUNT($0),EXPR$3=MAX($0),EXPR$4=MIN($0),agg#6=$SUM0($7),agg#7=COUNT($7),agg#8=$SUM0($1),agg#9=COUNT($1),EXPR$9=MAX($1),EXPR$10=MIN($1),agg#12=$SUM0($8),agg#13=COUNT($8),agg#14=$SUM0($2),agg#15=COUNT($2),EXPR$15=MAX($2),EXPR$16=MIN($2),agg#18=$SUM0($9),agg#19=COUNT($9),agg#20=$SUM0($3),agg#21=COUNT($3),EXPR$21=MAX($3),EXPR$22=MIN($3),agg#24=$SUM0($10),agg#25=COUNT($10),agg#26=$SUM0($4),agg#27=COUNT($4),EXPR$27=MAX($4),EXPR$28=MIN($4),agg#30=$SUM0($11),agg#31=COUNT($11),agg#32=$SUM0($5),agg#33=COUNT($5),EXPR$33=MAX($5),EXPR$34=MIN($5)), rowcount=100.0, cumulative cost={inf} rel#32:Subset#4.DATASET.[], best=null, importance=0.81 at org.apache.calcite.plan.volcano.RelSubset$CheapestPlanReplacer.visit(RelSubset.java:443) at org.apache.calcite.plan.volcano.RelSubset.buildCheapestPlan(RelSubset.java:293) at org.apache.calcite.plan.volcano.VolcanoPlanner.findBestExp(VolcanoPlanner.java:841) at org.apache.calcite.tools.Programs$RuleSetProgram.run(Programs.java:334) at org.apache.flink.api.table.BatchTableEnvironment.translate(BatchTableEnvironment.scala:253) I think what RelOptPlanner can't choose rule for which he needs to handle this query and can't apply any RelNode suitable for this sql query. For fix it need creating common rule which will contain rules for simple aggregate functions (sum, max, min, count) and new functions (stddev_pop, stddev_samp, etc) What do you think about it?
          Hide
          twalthr Timo Walther added a comment -

          The problem is that the rule translates not only into SUM and COUNT but also into SqlKind.SUM0/SqlSumEmptyIsZeroAggFunction which is not yet supported. This means we need to adapt the AggregateUtil to support this aggregation first. If I use the normal sum aggregator for it, I get the "Type NULL is not supported" exception. This is bug actually a null within a CASE expression should have a type.

          Show
          twalthr Timo Walther added a comment - The problem is that the rule translates not only into SUM and COUNT but also into SqlKind.SUM0 / SqlSumEmptyIsZeroAggFunction which is not yet supported. This means we need to adapt the AggregateUtil to support this aggregation first. If I use the normal sum aggregator for it, I get the "Type NULL is not supported" exception. This is bug actually a null within a CASE expression should have a type.
          Hide
          anmu Anton Mushin added a comment -

          Do you talking about new issue for support SqlKind.SUM0/SqlSumEmptyIsZeroAggFunction? or is it issue exist already? or do support SqlKind.SUM0 in this issue?

          Show
          anmu Anton Mushin added a comment - Do you talking about new issue for support SqlKind.SUM0/SqlSumEmptyIsZeroAggFunction ? or is it issue exist already? or do support SqlKind.SUM0 in this issue?
          Hide
          fhueske Fabian Hueske added a comment -

          Hi Anton Mushin,

          I think we can implement and add support for SqlKind.SUM0 as part of this issue.
          For that you have to implement a custom Sum0Aggregate which extends SumAggregate and overrides the prepare() method such that it does not initialize the aggregate with null but with 0 if the value is null.

          Next you have to fix the AggregateUtil and separate SqlSumEmptyIsZeroAggFunction from SqlSumAggFunction and initialize the new Sum0Aggregate and also allow for case SqlKind.SUM0 => true in DataSetAggregateRule.

          If I did not forget a place to add SUM0 support, that should do the trick.

          Show
          fhueske Fabian Hueske added a comment - Hi Anton Mushin , I think we can implement and add support for SqlKind.SUM0 as part of this issue. For that you have to implement a custom Sum0Aggregate which extends SumAggregate and overrides the prepare() method such that it does not initialize the aggregate with null but with 0 if the value is null . Next you have to fix the AggregateUtil and separate SqlSumEmptyIsZeroAggFunction from SqlSumAggFunction and initialize the new Sum0Aggregate and also allow for case SqlKind.SUM0 => true in DataSetAggregateRule . If I did not forget a place to add SUM0 support, that should do the trick.
          Hide
          anmu Anton Mushin added a comment -

          Hello Fabian Hueske, thanks for your answer!
          I added support SqlKind.SUM0 and implement custom Sum0Aggregate, but I'm still get

          DataSetCalc(select=[CAST(POWER(/(-($f0, /(*($f1, $f1), $f2)), $f2), 0.5)) AS EXPR$0, CASE(=($f2, 0), null, EXPR$1) AS EXPR$1])
            DataSetAggregate(select=[SUM($f6) AS $f0, SUM(_1) AS $f1, COUNT(_1) AS $f2, $SUM0(_1) AS EXPR$1])
              DataSetCalc(select=[_1, _2, _3, _4, _5, _6, *(_1, _1) AS $f6])
                DataSetScan(table=[[_DataSetTable_0]])
          
          org.apache.flink.api.table.TableException: Type NULL is not supported. Null values must have a supported type.
          
          	at org.apache.flink.api.table.FlinkTypeFactory$.toTypeInfo(FlinkTypeFactory.scala:138)
          	at org.apache.flink.api.table.codegen.CodeGenerator.visitLiteral(CodeGenerator.scala:553)
          

          and I don't understand where I getting CASE(=($f2, 0), null, EXPR$1) AS EXPR$1])

          Show
          anmu Anton Mushin added a comment - Hello Fabian Hueske , thanks for your answer! I added support SqlKind.SUM0 and implement custom Sum0Aggregate , but I'm still get DataSetCalc(select=[CAST(POWER(/(-($f0, /(*($f1, $f1), $f2)), $f2), 0.5)) AS EXPR$0, CASE(=($f2, 0), null, EXPR$1) AS EXPR$1]) DataSetAggregate(select=[SUM($f6) AS $f0, SUM(_1) AS $f1, COUNT(_1) AS $f2, $SUM0(_1) AS EXPR$1]) DataSetCalc(select=[_1, _2, _3, _4, _5, _6, *(_1, _1) AS $f6]) DataSetScan(table=[[_DataSetTable_0]]) org.apache.flink.api.table.TableException: Type NULL is not supported. Null values must have a supported type. at org.apache.flink.api.table.FlinkTypeFactory$.toTypeInfo(FlinkTypeFactory.scala:138) at org.apache.flink.api.table.codegen.CodeGenerator.visitLiteral(CodeGenerator.scala:553) and I don't understand where I getting CASE(=($f2, 0), null, EXPR$1) AS EXPR$1])
          Hide
          anmu Anton Mushin added a comment -

          I found rootcause

          problems occur only in query with SUM, if in query replace SUM to $SUM0 (or any aggregate function) then query works.
          At time execute AggregateReduceFunctionsRule#onMatch SUM must replace to $SUM0 here
          But it is not happening because the types in FLINK is nullable. ( in my case)

          I do not know how to solve this problem yet, please let me know if you have idea how resolve this problem.

          Show
          anmu Anton Mushin added a comment - I found rootcause problems occur only in query with SUM, if in query replace SUM to $SUM0 (or any aggregate function) then query works. At time execute AggregateReduceFunctionsRule#onMatch SUM must replace to $SUM0 here But it is not happening because the types in FLINK is nullable. ( in my case) I do not know how to solve this problem yet, please let me know if you have idea how resolve this problem.
          Hide
          anmu Anton Mushin added a comment -

          For correct calculate standard deviation/variance function need
          impliment support sqrt/power function for all data types.

          Show
          anmu Anton Mushin added a comment - For correct calculate standard deviation/variance function need impliment support sqrt/power function for all data types.
          Hide
          fhueske Fabian Hueske added a comment -

          Hi Anton Mushin, thanks for digging into this.

          Regarding the sqrt/power function, I guess you are aware of FLINK-4743 and PR #2686.
          So this issue should be fixed rather soon. In case this is blocking you, I'd rebase your code on top of the PR branch. We can later remove these commits once the original PR got merged.

          Regarding the Type NULL issue, are you still working on the branch you posted previously in this thread: BRANCH? I assume you get the exception when running one of tests, right?
          I will have a look and try to figure out what is going wrong there.

          Show
          fhueske Fabian Hueske added a comment - Hi Anton Mushin , thanks for digging into this. Regarding the sqrt/power function, I guess you are aware of FLINK-4743 and PR #2686 . So this issue should be fixed rather soon. In case this is blocking you, I'd rebase your code on top of the PR branch. We can later remove these commits once the original PR got merged. Regarding the Type NULL issue, are you still working on the branch you posted previously in this thread: BRANCH ? I assume you get the exception when running one of tests, right? I will have a look and try to figure out what is going wrong there.
          Hide
          anmu Anton Mushin added a comment -

          We can later remove these commits once the original PR got merged.

          I'm already merge this PR as patch

          Regarding the Type NULL issue, are you still working on the branch you posted previously in this thread: BRANCH?

          Yes, I working in this branch.

          I assume you get the exception when running one of tests, right?

          Yes, I have 2 tests: org.apache.flink.api.scala.batch.sql.AggregationsITCase#testStddevPopAggregateWithOtherAggreagteSUM0 and org.apache.flink.api.scala.batch.sql.AggregationsITCase#testStddevPopAggregateWithOtherAggreagteSUM.
          testStddevPopAggregateWithOtherAggreagteSUM0 is passing
          testStddevPopAggregateWithOtherAggreagteSUM is failing

          Show
          anmu Anton Mushin added a comment - We can later remove these commits once the original PR got merged. I'm already merge this PR as patch Regarding the Type NULL issue, are you still working on the branch you posted previously in this thread: BRANCH? Yes, I working in this branch . I assume you get the exception when running one of tests, right? Yes, I have 2 tests: org.apache.flink.api.scala.batch.sql.AggregationsITCase#testStddevPopAggregateWithOtherAggreagteSUM0 and org.apache.flink.api.scala.batch.sql.AggregationsITCase#testStddevPopAggregateWithOtherAggreagteSUM . testStddevPopAggregateWithOtherAggreagteSUM0 is passing testStddevPopAggregateWithOtherAggreagteSUM is failing
          Hide
          twalthr Timo Walther added a comment -

          I think the missing type for NULL is a Calcite issue. Usually, a NULL in a CASE(=($f2, 0), null, EXPR$1) should have the same type as EXPR$1. I will have a look at it today.

          Show
          twalthr Timo Walther added a comment - I think the missing type for NULL is a Calcite issue. Usually, a NULL in a CASE(=($f2, 0), null, EXPR$1) should have the same type as EXPR$1 . I will have a look at it today.
          Hide
          twalthr Timo Walther added a comment -

          I looked into the problem. If you take a look at AggregateReduceFunctionsRule (e.g. line 354), you see that the null literal is created with no type/the NULL type. We currently do not support this type. Either we replace this and similar lines with rexBuilder.makeNullLiteral(sumZeroRef.getType().getSqlTypeName()) to give it a type or we create a new type but I don't know how we want to represent it so far.

          Show
          twalthr Timo Walther added a comment - I looked into the problem. If you take a look at AggregateReduceFunctionsRule (e.g. line 354), you see that the null literal is created with no type/the NULL type. We currently do not support this type. Either we replace this and similar lines with rexBuilder.makeNullLiteral(sumZeroRef.getType().getSqlTypeName()) to give it a type or we create a new type but I don't know how we want to represent it so far.
          Hide
          twalthr Timo Walther added a comment -

          Any news on this Anton Mushin? You could solve this issue temporarily by doing it similar to FLINK-5144.

          Show
          twalthr Timo Walther added a comment - Any news on this Anton Mushin ? You could solve this issue temporarily by doing it similar to FLINK-5144 .
          Hide
          anmu Anton Mushin added a comment -

          Hi, Timo Walther
          I updated my branch after rebase on master.
          I still get org.apache.flink.table.api.TableException: Type NULL is not supported. Null values must have a supported type. for tests: org.apache.flink.table.api.scala.batch.table.AggregationsITCase#testPojoAggregation and org.apache.flink.table.api.scala.batch.sql.AggregationsITCase#testStddevPopAggregateWithOtherAggreagte

          Show
          anmu Anton Mushin added a comment - Hi, Timo Walther I updated my branch after rebase on master. I still get org.apache.flink.table.api.TableException: Type NULL is not supported. Null values must have a supported type. for tests: org.apache.flink.table.api.scala.batch.table.AggregationsITCase#testPojoAggregation and org.apache.flink.table.api.scala.batch.sql.AggregationsITCase#testStddevPopAggregateWithOtherAggreagte
          Hide
          twalthr Timo Walther added a comment -

          Hi Anton Mushin, I would suggest to open a PR for Apache Calcite (maybe discuss it first on the ML) for changing the AggregateReduceFunctionsRule (line 354) as I mentioned previously. If they accept this change, we can temporarily copy changed rule in the Flink code base until we update the Calcite version.

          Show
          twalthr Timo Walther added a comment - Hi Anton Mushin , I would suggest to open a PR for Apache Calcite (maybe discuss it first on the ML) for changing the AggregateReduceFunctionsRule (line 354) as I mentioned previously. If they accept this change, we can temporarily copy changed rule in the Flink code base until we update the Calcite version.
          Hide
          githubbot ASF GitHub Bot added a comment -

          GitHub user ex00 opened a pull request:

          https://github.com/apache/flink/pull/3260

          FLINK-4604 Add support for standard deviation/variance

          add rule for reduce standard deviation/variance functions

          Thanks for contributing to Apache Flink. Before you open your pull request, please take the following check list into consideration.
          If your changes take all of the items into account, feel free to open your pull request. For more information and/or questions please refer to the [How To Contribute guide](http://flink.apache.org/how-to-contribute.html).
          In addition to going through the list, please provide a meaningful description of your changes.

          • [x] General
          • The pull request references the related JIRA issue ("[FLINK-XXX] Jira title text")
          • The pull request addresses only one issue
          • Each commit in the PR has a meaningful commit message (including the JIRA id)
          • [x] Documentation
          • Documentation has been added for new functionality
          • Old documentation affected by the pull request has been updated
          • JavaDoc for public methods has been added
          • [x] Tests & Build
          • Functionality added by the pull request is covered by tests
          • `mvn clean verify` has been executed successfully locally or a Travis build has passed

          You can merge this pull request into a Git repository by running:

          $ git pull https://github.com/ex00/flink FLINK-4604

          Alternatively you can review and apply these changes as the patch at:

          https://github.com/apache/flink/pull/3260.patch

          To close this pull request, make a commit to your master/trunk branch
          with (at least) the following in the commit message:

          This closes #3260


          commit 38d99c4c6ccc8e8b1dbd2f83d9cef4eae3494f00
          Author: Anton Mushin <anton_mushin@epam.com>
          Date: 2017-02-03T10:06:49Z

          FLINK-4604 Add support for standard deviation/variance

          add rule for reduce standard deviation/variance functions


          Show
          githubbot ASF GitHub Bot added a comment - GitHub user ex00 opened a pull request: https://github.com/apache/flink/pull/3260 FLINK-4604 Add support for standard deviation/variance add rule for reduce standard deviation/variance functions Thanks for contributing to Apache Flink. Before you open your pull request, please take the following check list into consideration. If your changes take all of the items into account, feel free to open your pull request. For more information and/or questions please refer to the [How To Contribute guide] ( http://flink.apache.org/how-to-contribute.html ). In addition to going through the list, please provide a meaningful description of your changes. [x] General The pull request references the related JIRA issue (" [FLINK-XXX] Jira title text") The pull request addresses only one issue Each commit in the PR has a meaningful commit message (including the JIRA id) [x] Documentation Documentation has been added for new functionality Old documentation affected by the pull request has been updated JavaDoc for public methods has been added [x] Tests & Build Functionality added by the pull request is covered by tests `mvn clean verify` has been executed successfully locally or a Travis build has passed You can merge this pull request into a Git repository by running: $ git pull https://github.com/ex00/flink FLINK-4604 Alternatively you can review and apply these changes as the patch at: https://github.com/apache/flink/pull/3260.patch To close this pull request, make a commit to your master/trunk branch with (at least) the following in the commit message: This closes #3260 commit 38d99c4c6ccc8e8b1dbd2f83d9cef4eae3494f00 Author: Anton Mushin <anton_mushin@epam.com> Date: 2017-02-03T10:06:49Z FLINK-4604 Add support for standard deviation/variance add rule for reduce standard deviation/variance functions
          Hide
          anmu Anton Mushin added a comment -

          Hi everyone,
          Calcite 1.12 has been release: http://calcite.apache.org/news/2017/03/24/release-1.12.0/
          I will try update my PR with considering what CALCITE-1621 included in Calcite 1.12

          Show
          anmu Anton Mushin added a comment - Hi everyone, Calcite 1.12 has been release: http://calcite.apache.org/news/2017/03/24/release-1.12.0/ I will try update my PR with considering what CALCITE-1621 included in Calcite 1.12
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user ex00 commented on the issue:

          https://github.com/apache/flink/pull/3260

          Hello, I updated PR for calcite 1.12.

          Show
          githubbot ASF GitHub Bot added a comment - Github user ex00 commented on the issue: https://github.com/apache/flink/pull/3260 Hello, I updated PR for calcite 1.12.
          Hide
          anmu Anton Mushin added a comment -

          Hi Timo Walther,
          Could you look my PR for this issue, please?

          Show
          anmu Anton Mushin added a comment - Hi Timo Walther , Could you look my PR for this issue, please?
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user twalthr commented on the issue:

          https://github.com/apache/flink/pull/3260

          Thanks for the update @ex00. I will go through your changes and merge this tomorrow.

          Show
          githubbot ASF GitHub Bot added a comment - Github user twalthr commented on the issue: https://github.com/apache/flink/pull/3260 Thanks for the update @ex00. I will go through your changes and merge this tomorrow.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user asfgit closed the pull request at:

          https://github.com/apache/flink/pull/3260

          Show
          githubbot ASF GitHub Bot added a comment - Github user asfgit closed the pull request at: https://github.com/apache/flink/pull/3260
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user ex00 commented on the issue:

          https://github.com/apache/flink/pull/3260

          Thanks @twalthr!

          Show
          githubbot ASF GitHub Bot added a comment - Github user ex00 commented on the issue: https://github.com/apache/flink/pull/3260 Thanks @twalthr!
          Hide
          twalthr Timo Walther added a comment -

          Fixed in 1.3.0: 0af57fc1800d7430843a1e14bb70168bbd750389 & 72dfce40b76f31cdf54959577977a33a0082a7bb

          Show
          twalthr Timo Walther added a comment - Fixed in 1.3.0: 0af57fc1800d7430843a1e14bb70168bbd750389 & 72dfce40b76f31cdf54959577977a33a0082a7bb

            People

            • Assignee:
              anmu Anton Mushin
              Reporter:
              twalthr Timo Walther
            • Votes:
              0 Vote for this issue
              Watchers:
              7 Start watching this issue

              Dates

              • Created:
                Updated:
                Resolved:

                Development