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



##########
File path: 
flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/logical/windowingSpecs.scala
##########
@@ -0,0 +1,118 @@
+/*
+ * 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.logical
+
+import org.apache.flink.table.types.logical.LogicalType
+import org.apache.flink.table.types.logical.utils.LogicalTypeChecks
+import org.apache.flink.util.TimeUtils.formatWithHighestUnit
+
+import java.time.Duration
+import java.util.Objects
+
+/**
+ * Logical representation of a windowing strategy.
+ */
+sealed trait WindowingStrategy {
+  val window: WindowSpec
+  val timeAttributeType: LogicalType
+  val isRowtime: Boolean = 
LogicalTypeChecks.isRowtimeAttribute(timeAttributeType)
+  def toSummaryString(inputFieldNames: Array[String]): String
+}
+
+case class TimeAttributeWindowingStrategy(
+    timeAttribute: Int,
+    timeAttributeType: LogicalType,
+    window: WindowSpec)
+    extends WindowingStrategy {
+  override def toSummaryString(inputFieldNames: Array[String]): String = {
+    val windowing = s"time_col=[${inputFieldNames(timeAttribute)}]"
+    window.toSummaryString(windowing)
+  }
+}
+
+case class WindowAttachedWindowingStrategy(
+    windowStart: Int,
+    windowEnd: Int,
+    timeAttributeType: LogicalType,
+    window: WindowSpec)
+  extends WindowingStrategy {
+  override def toSummaryString(inputFieldNames: Array[String]): String = {
+    val windowing = s"win_start=[${inputFieldNames(windowStart)}], " +
+      s"win_end=[${inputFieldNames(windowEnd)}]"
+    window.toSummaryString(windowing)
+  }
+}
+
+// 
------------------------------------------------------------------------------------------------
+// Window specifications
+// 
------------------------------------------------------------------------------------------------
+
+/**
+ * Logical representation of a window specification.
+ */
+sealed trait WindowSpec {
+
+  def toSummaryString(windowing: String): String
+
+  def hashCode(): Int
+
+  def equals(obj: Any): Boolean
+}
+
+case class TumblingWindowSpec(size: Duration) extends WindowSpec {

Review comment:
       Does this pr plan to support `offset`?

##########
File path: 
flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/PushWindowTableFunctionIntoWindowAggregateRule.scala
##########
@@ -0,0 +1,152 @@
+/*
+ * 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.calcite.plan.RelOptRule.{any, operand}
+import org.apache.calcite.plan.{RelOptRule, RelOptRuleCall}
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.{RelCollations, RelNode}
+import org.apache.calcite.rex._
+import org.apache.calcite.util.ImmutableBitSet
+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 scala.collection.JavaConversions._
+import scala.collection.mutable.ArrayBuffer
+
+
+/**
+ * Planner rule that tries to push [[StreamPhysicalWindowTableFunction]] into a
+ * [[StreamPhysicalWindowAggregate]].
+ */
+class PushWindowTableFunctionIntoWindowAggregateRule
+  extends RelOptRule(

Review comment:
       This rule try to pull  WIndowTableFunction up to WindowAggregate instead 
of push into WindowAggregate,

##########
File path: 
flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/PushWindowTableFunctionIntoWindowAggregateRule.scala
##########
@@ -0,0 +1,152 @@
+/*
+ * 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.calcite.plan.RelOptRule.{any, operand}
+import org.apache.calcite.plan.{RelOptRule, RelOptRuleCall}
+import org.apache.calcite.rel.`type`.RelDataType

Review comment:
       remove unused import

##########
File path: 
flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/PushWindowTableFunctionIntoWindowAggregateRule.scala
##########
@@ -0,0 +1,152 @@
+/*
+ * 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.calcite.plan.RelOptRule.{any, operand}
+import org.apache.calcite.plan.{RelOptRule, RelOptRuleCall}
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.{RelCollations, RelNode}
+import org.apache.calcite.rex._
+import org.apache.calcite.util.ImmutableBitSet
+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 scala.collection.JavaConversions._
+import scala.collection.mutable.ArrayBuffer

Review comment:
       remove unused import

##########
File path: 
flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/trait/RelWindowProperties.java
##########
@@ -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.trait;
+
+import org.apache.flink.table.planner.plan.logical.WindowSpec;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.utils.LogicalTypeChecks;
+
+import org.apache.calcite.util.ImmutableBitSet;
+
+import javax.annotation.Nullable;
+
+import java.util.Objects;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/** It describes the information of window properties of a RelNode. */
+public class RelWindowProperties {
+
+    private final ImmutableBitSet windowStartColumns;
+    private final ImmutableBitSet windowEndColumns;
+    private final ImmutableBitSet windowTimeColumns;

Review comment:
       @wuchong ,  Do `windowStartColumns`/`windowEndColumns `/ 
`windowTimeColumns ` could only could be empty or contain 1 element, right?  Is 
there any possible that those `Immutablebitset`s contain more than one element? 

##########
File path: 
flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/sql/SqlCumulateTableFunction.java
##########
@@ -0,0 +1,72 @@
+/*
+ * 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.functions.sql;
+
+import org.apache.flink.shaded.guava18.com.google.common.collect.ImmutableList;
+
+import org.apache.calcite.sql.SqlCallBinding;
+import org.apache.calcite.sql.SqlOperator;
+
+/**
+ * SqlCumulateTableFunction implements an operator for cumulative.
+ *
+ * <p>It allows four parameters:
+ *
+ * <ol>
+ *   <li>a table
+ *   <li>a descriptor to provide a time attribute column name from the input 
table
+ *   <li>an interval parameter to specify the window size to increase.
+ *   <li>an interval parameter to specify the max length of window size
+ * </ol>
+ */
+public class SqlCumulateTableFunction extends SqlWindowTableFunction {

Review comment:
       Do you plan to support  window `offset` in this pr? 

##########
File path: 
flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/ExpandWindowTableFunctionTransposeRule.scala
##########
@@ -0,0 +1,298 @@
+/*
+ * 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.calcite.FlinkTypeFactory
+import org.apache.flink.table.planner.functions.sql.SqlWindowTableFunction
+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, 
StreamPhysicalExpand, 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.RelNode
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rex.{RexInputRef, RexLiteral, RexNode, RexProgram}
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+import scala.collection.mutable.ArrayBuffer
+
+/**
+ * This rule transposes [[StreamPhysicalExpand]] past 
[[StreamPhysicalWindowTableFunction]] to make
+ * [[PushWindowTableFunctionIntoWindowAggregateRule]] can match the rel tree 
pattern and optimize
+ * them into [[StreamPhysicalWindowAggregate]].
+ *
+ * Example:
+ *
+ * MyTable: a INT, c STRING, rowtime TIMESTAMP(3)
+ *
+ * SQL:
+ * {{{
+ * SELECT
+ *    window_start,
+ *    window_end,
+ *    count(distinct a),
+ *    count(distinct c)
+ * FROM TABLE(TUMBLE(TABLE MyTable, DESCRIPTOR(rowtime), INTERVAL '15' MINUTE))
+ * GROUP BY window_start, window_end
+ * }}}
+ *
+ * We will get part of the initial physical plan like following:
+ * {{{
+ * WindowAggregate(groupBy=[$f4, $f5], window=[TUMBLE(win_start=[window_start],
+ * win_end=[window_end], size=[15 min])], select=[$f4, $f5, COUNT(DISTINCT a) 
FILTER $g_1 AS $f2,
+ * COUNT(DISTINCT c) FILTER $g_2 AS $f3, start('w$) AS window_start, end('w$) 
AS window_end])
+ * +- Exchange(distribution=[hash[$f4, $f5]])
+ *    +- Calc(select=[window_start, window_end, a, c, $f4, $f5, =($e, 1) AS 
$g_1, =($e, 2) AS $g_2])
+ *       +- Expand(projects=[{window_start, window_end, a, c, $f4, null AS 
$f5, 1 AS $e},
+ *       {window_start, window_end, a, c, null AS $f4, $f5, 2 AS $e}])
+ *          +- Calc(select=[window_start, window_end, a, c,
+ *          MOD(HASH_CODE(a), 1024) AS $f4, MOD(HASH_CODE(c), 1024) AS $f5])
+ *             +- WindowTableFunction(window=[TUMBLE(time_col=[rowtime], 
size=[15 min])])
+ * }}}
+ *
+ * However, it can't match [[PushWindowTableFunctionIntoWindowAggregateRule]], 
because
+ * [[StreamPhysicalWindowTableFunction]] is not near 
[[StreamPhysicalWindowAggregate]].
+ * So we need to transpose [[StreamPhysicalExpand]] past 
[[StreamPhysicalWindowTableFunction]]
+ * to make the part of rel tree like this which can be matched by
+ * [[PushWindowTableFunctionIntoWindowAggregateRule]].
+ *

Review comment:
       Does this rule only aim to solve this pattern? If yes, could we just add 
a rule which convert the tree directly to the final equivalent instead of 
Intermediate `RelNode`  need post-process by 
`PushWindowTableFunctionIntoWindowAggregateRule `. 

##########
File path: 
flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/logical/windowingSpecs.scala
##########
@@ -0,0 +1,118 @@
+/*
+ * 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.logical
+
+import org.apache.flink.table.types.logical.LogicalType
+import org.apache.flink.table.types.logical.utils.LogicalTypeChecks
+import org.apache.flink.util.TimeUtils.formatWithHighestUnit
+
+import java.time.Duration
+import java.util.Objects
+
+/**
+ * Logical representation of a windowing strategy.
+ */
+sealed trait WindowingStrategy {
+  val window: WindowSpec
+  val timeAttributeType: LogicalType
+  val isRowtime: Boolean = 
LogicalTypeChecks.isRowtimeAttribute(timeAttributeType)
+  def toSummaryString(inputFieldNames: Array[String]): String
+}
+
+case class TimeAttributeWindowingStrategy(

Review comment:
       It's better to add comment on `TimeAttributeWindowingStrategy` and 
`WindowAttachedWindowingStrategy`. 
   And explain the difference between them.




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