xuyangzhong commented on a change in pull request #17632:
URL: https://github.com/apache/flink/pull/17632#discussion_r740708151



##########
File path: 
flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamPhysicalWindowDeduplicateRule.scala
##########
@@ -0,0 +1,95 @@
+/*
+ * 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.logical.WindowAttachedWindowingStrategy
+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.logical.FlinkLogicalRank
+import 
org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowDeduplicate
+import org.apache.flink.table.planner.plan.`trait`.FlinkRelDistribution
+import org.apache.flink.table.planner.plan.utils.{RankUtil, WindowUtil}
+
+import org.apache.calcite.plan.{RelOptRule, RelOptRuleCall}
+import org.apache.calcite.rel.convert.ConverterRule
+import org.apache.calcite.rel.RelNode
+
+/**
+ * Rule to convert a [[FlinkLogicalRank]] into a 
[[StreamPhysicalWindowDeduplicate]].
+ */
+class StreamPhysicalWindowDeduplicateRule
+  extends ConverterRule(
+    classOf[FlinkLogicalRank],
+    FlinkConventions.LOGICAL,
+    FlinkConventions.STREAM_PHYSICAL,
+    "StreamPhysicalWindowDeduplicateRule") {
+
+  override def matches(call: RelOptRuleCall): Boolean = {
+    val rank: FlinkLogicalRank = call.rel(0)
+
+    val fmq = FlinkRelMetadataQuery.reuseOrCreate(call.getMetadataQuery)
+    val windowProperties = fmq.getRelWindowProperties(rank.getInput)
+    val partitionKey = rank.partitionKey
+    WindowUtil.groupingContainsWindowStartEnd(partitionKey, windowProperties) 
&&
+      RankUtil.canConvertToDeduplicate(rank)
+  }
+
+  override def convert(rel: RelNode): RelNode = {
+    val rank: FlinkLogicalRank = rel.asInstanceOf[FlinkLogicalRank]
+    val fmq = 
FlinkRelMetadataQuery.reuseOrCreate(rel.getCluster.getMetadataQuery)
+    val relWindowProperties = fmq.getRelWindowProperties(rank.getInput)
+    val partitionKey = rank.partitionKey
+    val (startColumns, endColumns, _, newPartitionKey) =
+      WindowUtil.groupingExcludeWindowStartEndTimeColumns(partitionKey, 
relWindowProperties)
+    val requiredDistribution = if (!newPartitionKey.isEmpty) {
+      FlinkRelDistribution.hash(newPartitionKey.toArray, requireStrict = true)
+    } else {
+      FlinkRelDistribution.SINGLETON
+    }
+
+    val requiredTraitSet = rank.getCluster.getPlanner.emptyTraitSet()
+      .replace(requiredDistribution)
+      .replace(FlinkConventions.STREAM_PHYSICAL)
+    val providedTraitSet = 
rank.getTraitSet.replace(FlinkConventions.STREAM_PHYSICAL)
+    val newInput: RelNode = RelOptRule.convert(rank.getInput, requiredTraitSet)
+
+    val windowingStrategy = new WindowAttachedWindowingStrategy(
+      relWindowProperties.getWindowSpec,
+      relWindowProperties.getTimeAttributeType,
+      startColumns.toArray.head,
+      endColumns.toArray.head)
+
+    // order by timeIndicator desc ==> lastRow, otherwise is firstRow
+    val fieldCollation = rank.orderKey.getFieldCollations.get(0)

Review comment:
       ditto: Why always get the index '0' in the collections?

##########
File path: 
flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkChangelogModeInferenceProgram.scala
##########
@@ -210,7 +210,8 @@ class FlinkChangelogModeInferenceProgram extends 
FlinkOptimizeProgram[StreamOpti
         val providedTrait = new ModifyKindSetTrait(builder.build())
         createNewNode(window, children, providedTrait, requiredTrait, 
requester)
 
-      case _: StreamPhysicalWindowAggregate | _: StreamPhysicalWindowRank =>
+      case _: StreamPhysicalWindowAggregate | _: StreamPhysicalWindowRank |
+           _: StreamPhysicalWindowDeduplicate =>
         // WindowAggregate and WindowRank support insert-only in input

Review comment:
       tip: add the new info to the note  for consistency.

##########
File path: 
flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalWindowDeduplicate.scala
##########
@@ -0,0 +1,91 @@
+/*
+ * 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.nodes.physical.stream
+
+import org.apache.flink.table.planner.calcite.FlinkTypeFactory
+import org.apache.flink.table.planner.plan.logical.WindowingStrategy
+import 
org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecWindowDeduplicate
+import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty}
+import org.apache.flink.table.planner.plan.utils._
+
+import org.apache.calcite.plan.{RelOptCluster, RelTraitSet}
+import org.apache.calcite.rel._
+import org.apache.calcite.rel.`type`.RelDataType
+
+import java.util
+
+import scala.collection.JavaConverters._
+
+/**
+ * Stream physical RelNode which deduplicate on keys and keeps only first row 
or last row for each
+ * window. This node is an optimization of [[StreamPhysicalWindowRank]].
+ * Compared to [[StreamPhysicalWindowRank]], this node could access/write 
state with higher
+ * performance. The RelNode also requires PARTITION BY clause contains start 
and end columns of
+ * the windowing TVF.
+ */
+class StreamPhysicalWindowDeduplicate(
+    cluster: RelOptCluster,
+    traitSet: RelTraitSet,
+    inputRel: RelNode,
+    partitionKeys: Array[Int],
+    orderKey: Int,

Review comment:
       Could it happen that in the sql there are more than one field on 'order 
by' and then the order key is a collection?

##########
File path: 
flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalWindowDeduplicate.scala
##########
@@ -0,0 +1,91 @@
+/*
+ * 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.nodes.physical.stream
+
+import org.apache.flink.table.planner.calcite.FlinkTypeFactory
+import org.apache.flink.table.planner.plan.logical.WindowingStrategy
+import 
org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecWindowDeduplicate
+import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty}
+import org.apache.flink.table.planner.plan.utils._
+
+import org.apache.calcite.plan.{RelOptCluster, RelTraitSet}
+import org.apache.calcite.rel._
+import org.apache.calcite.rel.`type`.RelDataType
+
+import java.util
+
+import scala.collection.JavaConverters._
+
+/**
+ * Stream physical RelNode which deduplicate on keys and keeps only first row 
or last row for each
+ * window. This node is an optimization of [[StreamPhysicalWindowRank]].
+ * Compared to [[StreamPhysicalWindowRank]], this node could access/write 
state with higher
+ * performance. The RelNode also requires PARTITION BY clause contains start 
and end columns of
+ * the windowing TVF.
+ */
+class StreamPhysicalWindowDeduplicate(
+    cluster: RelOptCluster,
+    traitSet: RelTraitSet,
+    inputRel: RelNode,
+    partitionKeys: Array[Int],
+    orderKey: Int,

Review comment:
       Got it. Thanks for your contribution. ;)




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

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to