You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2020/07/10 15:22:08 UTC

[GitHub] [flink] leonardBang opened a new pull request #12873: [FLINK-15366][table-planner] Improve FlinkCalcMergeRule to merge calc nodes better

leonardBang opened a new pull request #12873:
URL: https://github.com/apache/flink/pull/12873


   ## What is the purpose of the change
   
   * This pull request Improve FlinkCalcMergeRule to merge calc nodes better.
   
   ## Brief change log
   
     -   Improve the  logic to judge two Calc node can merge or not: If two Calcs can merge, each bottomCalc's non-deterministic RexNode should appear at most once in topCalc's project fields and condition field.
   
   
   ## Verifying this change
   
   Add plan tests to cover, checked other plan tests are still ok.
   
   ## Does this pull request potentially affect one of the following parts:
   
     - Dependencies (does it add or upgrade a dependency): (no)
     - The public API, i.e., is any changed class annotated with `@Public(Evolving)`: (no)
     - The serializers: (no)
     - The runtime per-record code paths (performance sensitive): ( no)
     - Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Yarn/Mesos, ZooKeeper: (no)
     - The S3 file system connector: (no)
   
   ## Documentation
   
     - Does this pull request introduce a new feature? (no)
     - If yes, how is the feature documented? (not applicable / docs / JavaDocs / not documented)
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] godfreyhe commented on a change in pull request #12873: [FLINK-18548][table-planner] Improve FlinkCalcMergeRule to merge calc nodes better

Posted by GitBox <gi...@apache.org>.
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:
users@infra.apache.org



[GitHub] [flink] flinkbot commented on pull request #12873: [FLINK-18548][table-planner] Improve FlinkCalcMergeRule to merge calc nodes better

Posted by GitBox <gi...@apache.org>.
flinkbot commented on pull request #12873:
URL: https://github.com/apache/flink/pull/12873#issuecomment-656741832


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d348328658445fbc11b8a310f8f0fbd163acf308",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d348328658445fbc11b8a310f8f0fbd163acf308",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d348328658445fbc11b8a310f8f0fbd163acf308 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot commented on pull request #12873: [FLINK-18548][table-planner] Improve FlinkCalcMergeRule to merge calc nodes better

Posted by GitBox <gi...@apache.org>.
flinkbot commented on pull request #12873:
URL: https://github.com/apache/flink/pull/12873#issuecomment-656733286


   Thanks a lot for your contribution to the Apache Flink project. I'm the @flinkbot. I help the community
   to review your pull request. We will use this comment to track the progress of the review.
   
   
   ## Automated Checks
   Last check on commit d348328658445fbc11b8a310f8f0fbd163acf308 (Fri Jul 10 15:24:08 UTC 2020)
   
   **Warnings:**
    * No documentation files were touched! Remember to keep the Flink docs up to date!
   
   
   <sub>Mention the bot in a comment to re-run the automated checks.</sub>
   ## Review Progress
   
   * ❓ 1. The [description] looks good.
   * ❓ 2. There is [consensus] that the contribution should go into to Flink.
   * ❓ 3. Needs [attention] from.
   * ❓ 4. The change fits into the overall [architecture].
   * ❓ 5. Overall code [quality] is good.
   
   Please see the [Pull Request Review Guide](https://flink.apache.org/contributing/reviewing-prs.html) for a full explanation of the review process.<details>
    The Bot is tracking the review progress through labels. Labels are applied according to the order of the review items. For consensus, approval by a Flink committer of PMC member is required <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot approve description` to approve one or more aspects (aspects: `description`, `consensus`, `architecture` and `quality`)
    - `@flinkbot approve all` to approve all aspects
    - `@flinkbot approve-until architecture` to approve everything until `architecture`
    - `@flinkbot attention @username1 [@username2 ..]` to require somebody's attention
    - `@flinkbot disapprove architecture` to remove an approval you gave earlier
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] godfreyhe commented on a change in pull request #12873: [FLINK-18548][table-planner] Improve FlinkCalcMergeRule to merge calc nodes better

Posted by GitBox <gi...@apache.org>.
godfreyhe commented on a change in pull request #12873:
URL: https://github.com/apache/flink/pull/12873#discussion_r456230725



##########
File path: flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkCalcMergeRule.scala
##########
@@ -61,36 +61,43 @@ class FlinkCalcMergeRule(relBuilderFactory: RelBuilderFactory) extends RelOptRul
       return false
     }
 
-    // Each bottomCalc's non-deterministic RexNode should appear at most once in
-    // topCalc's project fields and condition field.
+    isMergeable(topCalc, bottomCalc)
+  }
+
+  /**
+   * Return two neighbouring [[Calc]] can merge into one [[Calc]] or not. If the two [[Calc]] can
+   * merge into one, each non-deterministic [[RexNode]] of bottom [[Calc]] should appear at most
+   * once in the project list and filter list of top [[Calc]].
+   */
+  private def isMergeable(topCalc: Calc, bottomCalc: Calc): Boolean = {
+    val topProgram = topCalc.getProgram
     val bottomProgram = bottomCalc.getProgram
-    val topProjectRexNodesInputs = topProgram.getProjectList
+    val topProjectInputIndices = 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
+    val topFilterInputIndices = if (topProgram.getCondition != null) {
+      InputFinder.bits(topProgram.expandLocalRef(topProgram.getCondition)).toArray
     } else {
       new Array[Int](0)
     }
 
-    val bottomRexList = bottomProgram.getProjectList
+    val bottomProjectList = bottomProgram.getProjectList
       .map(r => bottomProgram.expandLocalRef(r))
       .toArray
 
-    bottomRexList.zipWithIndex.forall {
-      case (rexNode: RexNode, index: Int) => {
+    bottomProjectList.zipWithIndex.forall {
+      case (project: RexNode, index: Int) => {
         var nonDeterministicRexRefCnt = 0
-        if (!RexUtil.isDeterministic(rexNode)) {
-          topProjectRexNodesInputs.foreach(list => list.foreach(
-            ref => if (ref == index) {
-              nonDeterministicRexRefCnt += 1
-            }))
-          topFilterRexNodesInputs.foreach(
-            ref => if (ref == index) {
-              nonDeterministicRexRefCnt += 1
-            })
+        if (!RexUtil.isDeterministic(project)) {
+          topProjectInputIndices.add(topFilterInputIndices)

Review comment:
       This line should move out-of `forall`, otherwise the `topProjectInputIndices` will contain many `topFilterInputIndices`s if there is multiple non-deterministic expression in bottom project.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12873: [FLINK-18548][table-planner] Improve FlinkCalcMergeRule to merge calc nodes better

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12873:
URL: https://github.com/apache/flink/pull/12873#issuecomment-656741832


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d348328658445fbc11b8a310f8f0fbd163acf308",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4410",
       "triggerID" : "d348328658445fbc11b8a310f8f0fbd163acf308",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24c77b9c54d326b4f6b51a716669df85d6f59a3f",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4545",
       "triggerID" : "24c77b9c54d326b4f6b51a716669df85d6f59a3f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d348328658445fbc11b8a310f8f0fbd163acf308 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4410) 
   * 24c77b9c54d326b4f6b51a716669df85d6f59a3f Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4545) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12873: [FLINK-18548][table-planner] Improve FlinkCalcMergeRule to merge calc nodes better

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12873:
URL: https://github.com/apache/flink/pull/12873#issuecomment-656741832


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d348328658445fbc11b8a310f8f0fbd163acf308",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4410",
       "triggerID" : "d348328658445fbc11b8a310f8f0fbd163acf308",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24c77b9c54d326b4f6b51a716669df85d6f59a3f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4545",
       "triggerID" : "24c77b9c54d326b4f6b51a716669df85d6f59a3f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85e3d0c3c21ddeaf780660db4909e37ad0672bcc",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "85e3d0c3c21ddeaf780660db4909e37ad0672bcc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8d9403461a2fcfdb61eaf8a37845ad760404b048",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4575",
       "triggerID" : "8d9403461a2fcfdb61eaf8a37845ad760404b048",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 85e3d0c3c21ddeaf780660db4909e37ad0672bcc UNKNOWN
   * 8d9403461a2fcfdb61eaf8a37845ad760404b048 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4575) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] godfreyhe commented on a change in pull request #12873: [FLINK-18548][table-planner] Improve FlinkCalcMergeRule to merge calc nodes better

Posted by GitBox <gi...@apache.org>.
godfreyhe commented on a change in pull request #12873:
URL: https://github.com/apache/flink/pull/12873#discussion_r456232970



##########
File path: flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkCalcMergeRule.scala
##########
@@ -86,12 +88,14 @@ class FlinkCalcMergeRule(relBuilderFactory: RelBuilderFactory) extends RelOptRul
       .map(r => bottomProgram.expandLocalRef(r))
       .toArray
 
+    topProjectInputIndices.add(topFilterInputIndices)
+    val topInputIndices = topProjectInputIndices

Review comment:
       topProjectInputIndices :+ topFilterInputIndices




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12873: [FLINK-18548][table-planner] Improve FlinkCalcMergeRule to merge calc nodes better

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12873:
URL: https://github.com/apache/flink/pull/12873#issuecomment-656741832


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d348328658445fbc11b8a310f8f0fbd163acf308",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4410",
       "triggerID" : "d348328658445fbc11b8a310f8f0fbd163acf308",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d348328658445fbc11b8a310f8f0fbd163acf308 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4410) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12873: [FLINK-18548][table-planner] Improve FlinkCalcMergeRule to merge calc nodes better

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12873:
URL: https://github.com/apache/flink/pull/12873#issuecomment-656741832


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d348328658445fbc11b8a310f8f0fbd163acf308",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4410",
       "triggerID" : "d348328658445fbc11b8a310f8f0fbd163acf308",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24c77b9c54d326b4f6b51a716669df85d6f59a3f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4545",
       "triggerID" : "24c77b9c54d326b4f6b51a716669df85d6f59a3f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 24c77b9c54d326b4f6b51a716669df85d6f59a3f Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4545) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12873: [FLINK-18548][table-planner] Improve FlinkCalcMergeRule to merge calc nodes better

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12873:
URL: https://github.com/apache/flink/pull/12873#issuecomment-656741832


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d348328658445fbc11b8a310f8f0fbd163acf308",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4410",
       "triggerID" : "d348328658445fbc11b8a310f8f0fbd163acf308",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d348328658445fbc11b8a310f8f0fbd163acf308 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4410) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] godfreyhe commented on a change in pull request #12873: [FLINK-18548][table-planner] Improve FlinkCalcMergeRule to merge calc nodes better

Posted by GitBox <gi...@apache.org>.
godfreyhe commented on a change in pull request #12873:
URL: https://github.com/apache/flink/pull/12873#discussion_r454853388



##########
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:
       nit: put them just one line ?




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12873: [FLINK-18548][table-planner] Improve FlinkCalcMergeRule to merge calc nodes better

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12873:
URL: https://github.com/apache/flink/pull/12873#issuecomment-656741832


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d348328658445fbc11b8a310f8f0fbd163acf308",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4410",
       "triggerID" : "d348328658445fbc11b8a310f8f0fbd163acf308",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24c77b9c54d326b4f6b51a716669df85d6f59a3f",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "24c77b9c54d326b4f6b51a716669df85d6f59a3f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d348328658445fbc11b8a310f8f0fbd163acf308 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4410) 
   * 24c77b9c54d326b4f6b51a716669df85d6f59a3f UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12873: [FLINK-18548][table-planner] Improve FlinkCalcMergeRule to merge calc nodes better

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12873:
URL: https://github.com/apache/flink/pull/12873#issuecomment-656741832


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d348328658445fbc11b8a310f8f0fbd163acf308",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4410",
       "triggerID" : "d348328658445fbc11b8a310f8f0fbd163acf308",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24c77b9c54d326b4f6b51a716669df85d6f59a3f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4545",
       "triggerID" : "24c77b9c54d326b4f6b51a716669df85d6f59a3f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85e3d0c3c21ddeaf780660db4909e37ad0672bcc",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "85e3d0c3c21ddeaf780660db4909e37ad0672bcc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8d9403461a2fcfdb61eaf8a37845ad760404b048",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4575",
       "triggerID" : "8d9403461a2fcfdb61eaf8a37845ad760404b048",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 24c77b9c54d326b4f6b51a716669df85d6f59a3f Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4545) 
   * 85e3d0c3c21ddeaf780660db4909e37ad0672bcc UNKNOWN
   * 8d9403461a2fcfdb61eaf8a37845ad760404b048 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4575) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12873: [FLINK-18548][table-planner] Improve FlinkCalcMergeRule to merge calc nodes better

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12873:
URL: https://github.com/apache/flink/pull/12873#issuecomment-656741832


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d348328658445fbc11b8a310f8f0fbd163acf308",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4410",
       "triggerID" : "d348328658445fbc11b8a310f8f0fbd163acf308",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24c77b9c54d326b4f6b51a716669df85d6f59a3f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4545",
       "triggerID" : "24c77b9c54d326b4f6b51a716669df85d6f59a3f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85e3d0c3c21ddeaf780660db4909e37ad0672bcc",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "85e3d0c3c21ddeaf780660db4909e37ad0672bcc",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 24c77b9c54d326b4f6b51a716669df85d6f59a3f Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4545) 
   * 85e3d0c3c21ddeaf780660db4909e37ad0672bcc UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] leonardBang commented on pull request #12873: [FLINK-18548][table-planner] Improve FlinkCalcMergeRule to merge calc nodes better

Posted by GitBox <gi...@apache.org>.
leonardBang commented on pull request #12873:
URL: https://github.com/apache/flink/pull/12873#issuecomment-659813009


   @godfreyhe Could you have a more look?


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12873: [FLINK-18548][table-planner] Improve FlinkCalcMergeRule to merge calc nodes better

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12873:
URL: https://github.com/apache/flink/pull/12873#issuecomment-656741832


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d348328658445fbc11b8a310f8f0fbd163acf308",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4410",
       "triggerID" : "d348328658445fbc11b8a310f8f0fbd163acf308",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24c77b9c54d326b4f6b51a716669df85d6f59a3f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4545",
       "triggerID" : "24c77b9c54d326b4f6b51a716669df85d6f59a3f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85e3d0c3c21ddeaf780660db4909e37ad0672bcc",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "85e3d0c3c21ddeaf780660db4909e37ad0672bcc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8d9403461a2fcfdb61eaf8a37845ad760404b048",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8d9403461a2fcfdb61eaf8a37845ad760404b048",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 24c77b9c54d326b4f6b51a716669df85d6f59a3f Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4545) 
   * 85e3d0c3c21ddeaf780660db4909e37ad0672bcc UNKNOWN
   * 8d9403461a2fcfdb61eaf8a37845ad760404b048 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12873: [FLINK-18548][table-planner] Improve FlinkCalcMergeRule to merge calc nodes better

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12873:
URL: https://github.com/apache/flink/pull/12873#issuecomment-656741832


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d348328658445fbc11b8a310f8f0fbd163acf308",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4410",
       "triggerID" : "d348328658445fbc11b8a310f8f0fbd163acf308",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24c77b9c54d326b4f6b51a716669df85d6f59a3f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4545",
       "triggerID" : "24c77b9c54d326b4f6b51a716669df85d6f59a3f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85e3d0c3c21ddeaf780660db4909e37ad0672bcc",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "85e3d0c3c21ddeaf780660db4909e37ad0672bcc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8d9403461a2fcfdb61eaf8a37845ad760404b048",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4575",
       "triggerID" : "8d9403461a2fcfdb61eaf8a37845ad760404b048",
       "triggerType" : "PUSH"
     }, {
       "hash" : "228c4360654f070e4730f60be2c4173134064de0",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4932",
       "triggerID" : "228c4360654f070e4730f60be2c4173134064de0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4b5b9ee889681cf25097c4cf58023c028a72185c",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "4b5b9ee889681cf25097c4cf58023c028a72185c",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 85e3d0c3c21ddeaf780660db4909e37ad0672bcc UNKNOWN
   * 228c4360654f070e4730f60be2c4173134064de0 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4932) 
   * 4b5b9ee889681cf25097c4cf58023c028a72185c UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] wuchong merged pull request #12873: [FLINK-18548][table-planner] Improve FlinkCalcMergeRule to merge calc nodes better

Posted by GitBox <gi...@apache.org>.
wuchong merged pull request #12873:
URL: https://github.com/apache/flink/pull/12873


   


----------------------------------------------------------------
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:
users@infra.apache.org