cloud-fan commented on code in PR #38799:
URL: https://github.com/apache/spark/pull/38799#discussion_r1046820898


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowGroupLimitExec.scala:
##########
@@ -0,0 +1,251 @@
+/*
+ * 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.spark.sql.execution.window
+
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{Ascending, Attribute, 
DenseRank, Expression, Rank, RowNumber, SortOrder, UnsafeProjection, UnsafeRow}
+import org.apache.spark.sql.catalyst.expressions.codegen.GenerateOrdering
+import org.apache.spark.sql.catalyst.plans.physical.{AllTuples, 
ClusteredDistribution, Distribution, Partitioning}
+import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode}
+
+sealed trait WindowGroupLimitMode
+
+case object Partial extends WindowGroupLimitMode
+
+case object Final extends WindowGroupLimitMode
+
+/**
+ * This operator is designed to filter out unnecessary rows before WindowExec
+ * for top-k computation.
+ * @param partitionSpec Should be the same as [[WindowExec#partitionSpec]]
+ * @param orderSpec Should be the same as [[WindowExec#orderSpec]]
+ * @param rankLikeFunction The function to compute row rank, should be 
RowNumber/Rank/DenseRank.
+ */
+case class WindowGroupLimitExec(
+    partitionSpec: Seq[Expression],
+    orderSpec: Seq[SortOrder],
+    rankLikeFunction: Expression,
+    limit: Int,
+    mode: WindowGroupLimitMode,
+    child: SparkPlan) extends UnaryExecNode {
+
+  override def output: Seq[Attribute] = child.output
+
+  override def requiredChildDistribution: Seq[Distribution] = mode match {
+    case Partial => super.requiredChildDistribution
+    case Final =>
+      if (partitionSpec.isEmpty) {
+        AllTuples :: Nil
+      } else {
+        ClusteredDistribution(partitionSpec) :: Nil
+      }
+  }
+
+  override def requiredChildOrdering: Seq[Seq[SortOrder]] =
+    Seq(partitionSpec.map(SortOrder(_, Ascending)) ++ orderSpec)
+
+  override def outputOrdering: Seq[SortOrder] = child.outputOrdering
+
+  override def outputPartitioning: Partitioning = child.outputPartitioning
+
+  protected override def doExecute(): RDD[InternalRow] = rankLikeFunction 
match {
+    case _: RowNumber =>
+      child.execute().mapPartitions(SimpleGroupLimitIterator(partitionSpec, 
output, _, limit))
+    case _: Rank =>
+      child.execute().mapPartitions(
+        RankGroupLimitIterator(partitionSpec, output, _, orderSpec, limit))
+    case _: DenseRank =>
+      child.execute().mapPartitions(
+        DenseRankGroupLimitIterator(partitionSpec, output, _, orderSpec, 
limit))
+  }
+
+  override protected def withNewChildInternal(newChild: SparkPlan): 
WindowGroupLimitExec =
+    copy(child = newChild)
+}
+
+abstract class WindowIterator extends Iterator[InternalRow] {
+
+  def partitionSpec: Seq[Expression]
+
+  def output: Seq[Attribute]
+
+  def stream: Iterator[InternalRow]
+
+  val grouping = UnsafeProjection.create(partitionSpec, output)
+
+  // Manage the stream and the grouping.
+  var nextRow: UnsafeRow = null
+  var nextGroup: UnsafeRow = null
+  var nextRowAvailable: Boolean = false
+  protected[this] def fetchNextRow(): Unit = {
+    nextRowAvailable = stream.hasNext
+    if (nextRowAvailable) {
+      nextRow = stream.next().asInstanceOf[UnsafeRow]
+      nextGroup = grouping(nextRow)
+    } else {
+      nextRow = null
+      nextGroup = null
+    }
+  }
+  fetchNextRow()
+
+  // Whether or not the rank exceeding the window group limit value.
+  def exceedingLimit(): Boolean
+
+  // Increase the rank value.
+  def increaseRank(): Unit
+
+  // Clear the rank value.
+  def clearRank(): Unit
+
+  var bufferIterator: Iterator[InternalRow] = _
+
+  private[this] def fetchNextPartition(): Unit = {

Review Comment:
   ```suggestion
     private[this] def fetchNextGroup(): Unit = {
   ```



-- 
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: reviews-unsubscr...@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org

Reply via email to