godfreyhe commented on a change in pull request #12873: URL: https://github.com/apache/flink/pull/12873#discussion_r454852639
########## File path: flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkCalcMergeRule.scala ########## @@ -60,9 +61,40 @@ class FlinkCalcMergeRule(relBuilderFactory: RelBuilderFactory) extends RelOptRul return false } - // Don't merge Calcs which contain non-deterministic expr - topProgram.getExprList.forall(RexUtil.isDeterministic) && - bottomCalc.getProgram.getExprList.forall(RexUtil.isDeterministic) + // Each bottomCalc's non-deterministic RexNode should appear at most once in + // topCalc's project fields and condition field. + val bottomProgram = bottomCalc.getProgram + val topProjectRexNodesInputs = topProgram.getProjectList + .map(r => topProgram.expandLocalRef(r)) + .map(r => InputFinder.bits(r).toArray) + + val topFilterRexNodesInputs = if (topProgram.getCondition != null) { Review comment: nit: rename to `topFilterInputIndices ` ########## File path: flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkCalcMergeRule.scala ########## @@ -60,9 +61,40 @@ class FlinkCalcMergeRule(relBuilderFactory: RelBuilderFactory) extends RelOptRul return false } - // Don't merge Calcs which contain non-deterministic expr - topProgram.getExprList.forall(RexUtil.isDeterministic) && - bottomCalc.getProgram.getExprList.forall(RexUtil.isDeterministic) + // Each bottomCalc's non-deterministic RexNode should appear at most once in + // topCalc's project fields and condition field. + val bottomProgram = bottomCalc.getProgram + val topProjectRexNodesInputs = topProgram.getProjectList + .map(r => topProgram.expandLocalRef(r)) + .map(r => InputFinder.bits(r).toArray) + + val topFilterRexNodesInputs = if (topProgram.getCondition != null) { + InputFinder.bits(topProgram.expandLocalRef(topProgram.getCondition)) + .toArray + } else { + new Array[Int](0) + } + + val bottomRexList = bottomProgram.getProjectList + .map(r => bottomProgram.expandLocalRef(r)) + .toArray + + bottomRexList.zipWithIndex.forall { + case (rexNode: RexNode, index: Int) => { Review comment: nit: rename `rexNode` to `project` ########## File path: flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkCalcMergeRule.scala ########## @@ -60,9 +61,40 @@ class FlinkCalcMergeRule(relBuilderFactory: RelBuilderFactory) extends RelOptRul return false } - // Don't merge Calcs which contain non-deterministic expr - topProgram.getExprList.forall(RexUtil.isDeterministic) && - bottomCalc.getProgram.getExprList.forall(RexUtil.isDeterministic) + // Each bottomCalc's non-deterministic RexNode should appear at most once in + // topCalc's project fields and condition field. + val bottomProgram = bottomCalc.getProgram + val topProjectRexNodesInputs = topProgram.getProjectList Review comment: nit: rename to `topProjectInputIndices` ? ########## File path: flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkCalcMergeRule.scala ########## @@ -60,9 +61,40 @@ class FlinkCalcMergeRule(relBuilderFactory: RelBuilderFactory) extends RelOptRul return false } - // Don't merge Calcs which contain non-deterministic expr - topProgram.getExprList.forall(RexUtil.isDeterministic) && - bottomCalc.getProgram.getExprList.forall(RexUtil.isDeterministic) + // Each bottomCalc's non-deterministic RexNode should appear at most once in + // topCalc's project fields and condition field. + val bottomProgram = bottomCalc.getProgram + val topProjectRexNodesInputs = topProgram.getProjectList + .map(r => topProgram.expandLocalRef(r)) + .map(r => InputFinder.bits(r).toArray) + + val topFilterRexNodesInputs = if (topProgram.getCondition != null) { + InputFinder.bits(topProgram.expandLocalRef(topProgram.getCondition)) + .toArray + } else { + new Array[Int](0) + } + + val bottomRexList = bottomProgram.getProjectList Review comment: nit: rename to `bottomProjectList` ########## File path: flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkCalcMergeRule.scala ########## @@ -60,9 +61,40 @@ class FlinkCalcMergeRule(relBuilderFactory: RelBuilderFactory) extends RelOptRul return false } - // Don't merge Calcs which contain non-deterministic expr - topProgram.getExprList.forall(RexUtil.isDeterministic) && - bottomCalc.getProgram.getExprList.forall(RexUtil.isDeterministic) + // Each bottomCalc's non-deterministic RexNode should appear at most once in + // topCalc's project fields and condition field. + val bottomProgram = bottomCalc.getProgram + val topProjectRexNodesInputs = topProgram.getProjectList + .map(r => topProgram.expandLocalRef(r)) + .map(r => InputFinder.bits(r).toArray) + + val topFilterRexNodesInputs = if (topProgram.getCondition != null) { + InputFinder.bits(topProgram.expandLocalRef(topProgram.getCondition)) + .toArray + } else { + new Array[Int](0) + } + + val bottomRexList = bottomProgram.getProjectList + .map(r => bottomProgram.expandLocalRef(r)) + .toArray + + bottomRexList.zipWithIndex.forall { + case (rexNode: RexNode, index: Int) => { + var nonDeterministicRexRefCnt = 0 + if (!RexUtil.isDeterministic(rexNode)) { + topProjectRexNodesInputs.foreach(list => list.foreach( Review comment: rename `list` to `indices` ? ########## File path: flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkCalcMergeRuleTest.scala ########## @@ -82,4 +82,35 @@ class FlinkCalcMergeRuleTest extends TableTestBase { val sqlQuery = "SELECT a FROM (SELECT a FROM MyTable) t WHERE random_udf(a) > 10" util.verifyPlan(sqlQuery) } + + @Test + def testCalcMergeWithNestedNonDeterministicExpr(): Unit = { + util.addFunction("random_udf", new NonDeterministicUdf) + val sqlQuery = "SELECT random_udf(a1) as a2 FROM (SELECT random_udf(a) as" + + " a1, b FROM MyTable) t WHERE b > 10" + util.verifyPlan(sqlQuery) + } + + @Test + def testCalcMergeWithTopMultiNonDeterministicExpr(): Unit = { + util.addFunction("random_udf", new NonDeterministicUdf) + val sqlQuery = "SELECT random_udf(a1) as a2, random_udf(a1) as a3 FROM" + + " (SELECT random_udf(a) as a1, b FROM MyTable) t WHERE b > 10" + util.verifyPlan(sqlQuery) + } + + @Test + def testCalcMergeWithBottomMultiNonDeterministicExpr(): Unit = { + util.addFunction("random_udf", new NonDeterministicUdf) + val sqlQuery = "SELECT a1, b2 FROM" + + " (SELECT random_udf(a) as a1, random_udf(b) as b2, c FROM MyTable) t WHERE c > 10" + util.verifyPlan(sqlQuery) + } + + @Test + def testCalcMergeWithoutInnerNonDeterministicExpr(): Unit = { + util.addFunction("random_udf", new NonDeterministicUdf) + val sqlQuery = "SELECT a, c FROM (SELECT a, random_udf(a) as a1, c FROM MyTable) t WHERE c > 10" + util.verifyPlan(sqlQuery) + } Review comment: please add a case about: top filter references a bottom non-deterministic project ########## File path: flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkCalcMergeRuleTest.scala ########## @@ -82,4 +82,35 @@ class FlinkCalcMergeRuleTest extends TableTestBase { val sqlQuery = "SELECT a FROM (SELECT a FROM MyTable) t WHERE random_udf(a) > 10" util.verifyPlan(sqlQuery) } + + @Test + def testCalcMergeWithNestedNonDeterministicExpr(): Unit = { + util.addFunction("random_udf", new NonDeterministicUdf) Review comment: move this common line into `setup` method ########## File path: flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkCalcMergeRule.scala ########## @@ -60,9 +61,40 @@ class FlinkCalcMergeRule(relBuilderFactory: RelBuilderFactory) extends RelOptRul return false } - // Don't merge Calcs which contain non-deterministic expr - topProgram.getExprList.forall(RexUtil.isDeterministic) && - bottomCalc.getProgram.getExprList.forall(RexUtil.isDeterministic) + // Each bottomCalc's non-deterministic RexNode should appear at most once in + // topCalc's project fields and condition field. + val bottomProgram = bottomCalc.getProgram + val topProjectRexNodesInputs = topProgram.getProjectList + .map(r => topProgram.expandLocalRef(r)) + .map(r => InputFinder.bits(r).toArray) + + val topFilterRexNodesInputs = if (topProgram.getCondition != null) { + InputFinder.bits(topProgram.expandLocalRef(topProgram.getCondition)) + .toArray Review comment: put them just one line ? ########## File path: flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkCalcMergeRule.scala ########## @@ -60,9 +61,40 @@ class FlinkCalcMergeRule(relBuilderFactory: RelBuilderFactory) extends RelOptRul return false } - // Don't merge Calcs which contain non-deterministic expr - topProgram.getExprList.forall(RexUtil.isDeterministic) && - bottomCalc.getProgram.getExprList.forall(RexUtil.isDeterministic) + // Each bottomCalc's non-deterministic RexNode should appear at most once in Review comment: extract these logic into another method? and please update the javadoc ########## File path: flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkCalcMergeRule.scala ########## @@ -60,9 +61,40 @@ class FlinkCalcMergeRule(relBuilderFactory: RelBuilderFactory) extends RelOptRul return false } - // Don't merge Calcs which contain non-deterministic expr - topProgram.getExprList.forall(RexUtil.isDeterministic) && - bottomCalc.getProgram.getExprList.forall(RexUtil.isDeterministic) + // Each bottomCalc's non-deterministic RexNode should appear at most once in + // topCalc's project fields and condition field. + val bottomProgram = bottomCalc.getProgram + val topProjectRexNodesInputs = topProgram.getProjectList + .map(r => topProgram.expandLocalRef(r)) + .map(r => InputFinder.bits(r).toArray) + + val topFilterRexNodesInputs = if (topProgram.getCondition != null) { + InputFinder.bits(topProgram.expandLocalRef(topProgram.getCondition)) + .toArray + } else { + new Array[Int](0) + } + + val bottomRexList = bottomProgram.getProjectList + .map(r => bottomProgram.expandLocalRef(r)) + .toArray + + bottomRexList.zipWithIndex.forall { + case (rexNode: RexNode, index: Int) => { + var nonDeterministicRexRefCnt = 0 + if (!RexUtil.isDeterministic(rexNode)) { + topProjectRexNodesInputs.foreach(list => list.foreach( + ref => if (ref == index) { + nonDeterministicRexRefCnt += 1 + })) + topFilterRexNodesInputs.foreach( Review comment: combine `topProjectRexNodesInputs` and `topFilterRexNodesInputs ` into one list, then this `foreach` can be removed ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org