wuchong commented on a change in pull request #14905:
URL: https://github.com/apache/flink/pull/14905#discussion_r580107506



##########
File path: 
flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/PullUpWindowTableFunctionIntoWindowAggregateRule.scala
##########
@@ -0,0 +1,149 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.planner.plan.rules.physical.stream
+
+import org.apache.flink.table.planner.plan.`trait`.FlinkRelDistribution
+import 
org.apache.flink.table.planner.plan.logical.TimeAttributeWindowingStrategy
+import org.apache.flink.table.planner.plan.metadata.FlinkRelMetadataQuery
+import org.apache.flink.table.planner.plan.nodes.FlinkConventions
+import 
org.apache.flink.table.planner.plan.nodes.physical.stream.{StreamPhysicalCalc, 
StreamPhysicalExchange, StreamPhysicalWindowAggregate, 
StreamPhysicalWindowTableFunction}
+import org.apache.flink.table.planner.plan.utils.WindowUtil
+import 
org.apache.flink.table.planner.plan.utils.WindowUtil.buildNewProgramWithoutWindowColumns
+
+import org.apache.calcite.plan.RelOptRule.{any, operand}
+import org.apache.calcite.plan.{RelOptRule, RelOptRuleCall}
+import org.apache.calcite.rel.{RelCollations, RelNode}
+import org.apache.calcite.util.ImmutableBitSet
+
+import scala.collection.JavaConversions._
+
+/**
+ * Planner rule that tries to pull up [[StreamPhysicalWindowTableFunction]] 
into a
+ * [[StreamPhysicalWindowAggregate]].
+ */
+class PullUpWindowTableFunctionIntoWindowAggregateRule
+  extends RelOptRule(
+    operand(classOf[StreamPhysicalWindowAggregate],
+      operand(classOf[StreamPhysicalExchange],
+        operand(classOf[StreamPhysicalCalc],
+          operand(classOf[StreamPhysicalWindowTableFunction], any())))),
+    "PullUpWindowTableFunctionIntoWindowAggregateRule"){
+
+  override def matches(call: RelOptRuleCall): Boolean = {
+    val windowAgg: StreamPhysicalWindowAggregate = call.rel(0)
+    val calc: StreamPhysicalCalc = call.rel(2)
+    val fmq = 
FlinkRelMetadataQuery.reuseOrCreate(windowAgg.getCluster.getMetadataQuery)
+
+    // condition and projection of Calc shouldn't contain calls on window 
columns,
+    // otherwise, we can't transpose WindowTVF and Calc
+    if (WindowUtil.calcContainsCallsOnWindowColumns(calc, fmq)) {
+      return false
+    }
+
+    val aggInputWindowProps = fmq.getRelWindowProperties(calc).getWindowColumns
+    // aggregate call shouldn't be on window columns
+    // TODO: this can be supported in the future by referencing them as a 
RexFieldVariable
+    windowAgg.aggCalls.forall { call =>
+      
aggInputWindowProps.intersect(ImmutableBitSet.of(call.getArgList)).isEmpty
+    }
+  }
+
+  override def onMatch(call: RelOptRuleCall): Unit = {
+    val windowAgg: StreamPhysicalWindowAggregate = call.rel(0)
+    val calc: StreamPhysicalCalc = call.rel(2)
+    val windowTVF: StreamPhysicalWindowTableFunction = call.rel(3)
+    val fmq = 
FlinkRelMetadataQuery.reuseOrCreate(windowAgg.getCluster.getMetadataQuery)
+    val cluster = windowAgg.getCluster
+    val input = windowTVF.getInput

Review comment:
       I think it's fine to keep them, because they almost are all assignment 
statements. 




----------------------------------------------------------------
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


Reply via email to