jnh5y commented on code in PR #25752:
URL: https://github.com/apache/flink/pull/25752#discussion_r1873947221


##########
flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamLogicalOptimizeSelectDistinctRule.scala:
##########
@@ -0,0 +1,140 @@
+/*
+ * 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.JList
+import org.apache.flink.table.planner.calcite.FlinkTypeFactory
+import org.apache.flink.table.planner.plan.nodes.FlinkConventions
+import 
org.apache.flink.table.planner.plan.nodes.logical.{FlinkLogicalAggregate, 
FlinkLogicalCalc, FlinkLogicalJoin, FlinkLogicalRank}
+import 
org.apache.flink.table.planner.plan.nodes.physical.stream.{StreamPhysicalIntervalJoin,
 StreamPhysicalRank, StreamPhysicalTemporalSort}
+import org.apache.flink.table.planner.plan.utils.{RankProcessStrategy, 
RankUtil, WindowJoinUtil}
+import org.apache.flink.table.runtime.operators.rank.{ConstantRankRange, 
RankType}
+import org.apache.flink.table.types.logical.IntType
+
+import org.apache.calcite.plan.{RelOptRule, RelOptRuleCall}
+import org.apache.calcite.plan.RelOptRule.{any, operand}
+import org.apache.calcite.rel.`type`.{RelDataType, RelDataTypeField, 
RelDataTypeFieldImpl, RelDataTypeSystem}
+import org.apache.calcite.rel.{RelCollation, RelCollations, RelFieldCollation}
+import org.apache.calcite.rel.RelNode
+import org.apache.calcite.rel.convert.ConverterRule.Config
+import org.apache.calcite.rel.core.JoinRelType
+import org.apache.calcite.rel.hint.RelHint
+import org.apache.calcite.rel.logical.LogicalProject
+import org.apache.calcite.rex.{RexInputRef, RexNode, RexProgram}
+import org.apache.calcite.util.ImmutableBitSet
+
+import java.util
+import java.util.Collections
+
+import scala.collection.convert.ImplicitConversions.`iterable AsScalaIterable`
+
+/**
+ * Rule that matches [[FlinkLogicalAggregate]], and converts it to 
[[FlinkLogicalRank]] in the case
+ * of SELECT DISTINCT queries.
+ *
+ * e.g. {SELECT DISTINCT a, b, c;} will be converted to [[FlinkLogicalRank]] 
instead of
+ * [[FlinkLogicalAggregate]] in rowtime.
+ */
+class StreamLogicalOptimizeSelectDistinctRule

Review Comment:
   @yiyutian1 and I worked on this together and we started with a Scala 
example. 
   
   Given that we are working on migrating the Scala rules to Java, could you 
take a look at migrating this rule to Java?



##########
flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamPhysicalGroupAggregateRule.scala:
##########
@@ -38,7 +38,6 @@ class StreamPhysicalGroupAggregateRule(config: Config) 
extends ConverterRule(con
 
   override def matches(call: RelOptRuleCall): Boolean = {
     val agg: FlinkLogicalAggregate = call.rel(0)
-

Review Comment:
   Stray change.



-- 
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: issues-unsubscr...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to