You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by we...@apache.org on 2023/02/21 07:09:06 UTC

[spark] branch master updated: [SPARK-37099][SQL] Introduce the group limit of Window for rank-based filter to optimize top-k computation

This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
     new 0e8a20e6da1 [SPARK-37099][SQL] Introduce the group limit of Window for rank-based filter to optimize top-k computation
0e8a20e6da1 is described below

commit 0e8a20e6da13e10ea9f0b47a8cd132c226cc3282
Author: Jiaan Geng <be...@163.com>
AuthorDate: Tue Feb 21 15:08:48 2023 +0800

    [SPARK-37099][SQL] Introduce the group limit of Window for rank-based filter to optimize top-k computation
    
    ### What changes were proposed in this pull request?
    introduce a new node `WindowGroupLimit` to filter out unnecessary rows based on rank computed on partial dataset.
    
    it supports following pattern:
    ```
    SELECT (... (row_number|rank|dense_rank)()
        OVER (
    PARTITION BY ...
    ORDER BY  ... ) AS rn)
    WHERE rn (==|<|<=) k
            AND other conditions
    ```
    
    For these three rank-like functions (row_number|rank|dense_rank), the rank of a key computed on partial dataset always <= its final rank computed on the whole dataset,so we can safely discard rows with partial rank > k, anywhere.
    
    This PR adds a per-window-group limit before and after the shuffle to reduce the input data of window processing.
    
    More specifically, the before-shuffle per-window-group limit:
    
    1. adds some row comparison to determine window group boundaries and rank values.
    
    2. applies per-window-group limit to reduce the data size of shuffle, and all the downstream operators.
    
    This is beneficial, assuming the per-window-group data size is large. Otherwise, the method to determine window group boundaries is pure overhead. The config `spark.sql.window.group.limit.threshold` could avoid the overhead if the per-window-group data size is small enough.
    
    The after-shuffle per-window-group limit just applies a per-window-group limit to reduce the data size of window processing. This is very cheap as it only needs to iterate the sorted data (window operator needs to sort the input) once and do some row comparison to determine group boundaries and rank values. It's more efficient to merge it into the window operator, but probably doesn't worth it as the overhead is small.
    
    This PR also take over some functions from https://github.com/apache/spark/pull/34367.
    
    ### Why are the changes needed?
    
    1. reduce the shuffle write
    2. solve skewed-window problem.
    3. improve the performance and TPC-DS.
    
    ### Does this PR introduce _any_ user-facing change?
    'No'.
    Just update the inner implementation and add a new config.
    
    ### How was this patch tested?
    
    1. new test suites
    2. new micro benchmark
    ```
    Java HotSpot(TM) 64-Bit Server VM 1.8.0_311-b11 on Mac OS X 10.16
    Intel(R) Core(TM) i7-9750H CPU  2.60GHz
    Benchmark Top-K:                                                 Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
    -----------------------------------------------------------------------------------------------------------------------------------------------
    ROW_NUMBER (PARTITION: , WindowGroupLimit: false)                        10711          11068         363          2.0         510.8       1.0X
    ROW_NUMBER (PARTITION: , WindowGroupLimit: true)                          2611           2929         210          8.0         124.5       4.1X
    ROW_NUMBER (PARTITION: PARTITION BY b, WindowGroupLimit: false)          23829          24085         178          0.9        1136.2       0.4X
    ROW_NUMBER (PARTITION: PARTITION BY b, WindowGroupLimit: true)            6381           6515          97          3.3         304.3       1.7X
    RANK (PARTITION: , WindowGroupLimit: false)                              11459          11798         223          1.8         546.4       0.9X
    RANK (PARTITION: , WindowGroupLimit: true)                                2588           2837         162          8.1         123.4       4.1X
    RANK (PARTITION: PARTITION BY b, WindowGroupLimit: false)                24560          24707         108          0.9        1171.1       0.4X
    RANK (PARTITION: PARTITION BY b, WindowGroupLimit: true)                  6395           6530         109          3.3         304.9       1.7X
    DENSE_RANK (PARTITION: , WindowGroupLimit: false)                        11563          11740         124          1.8         551.3       0.9X
    DENSE_RANK (PARTITION: , WindowGroupLimit: true)                          2563           2819         168          8.2         122.2       4.2X
    DENSE_RANK (PARTITION: PARTITION BY b, WindowGroupLimit: false)          24482          24578          64          0.9        1167.4       0.4X
    DENSE_RANK (PARTITION: PARTITION BY b, WindowGroupLimit: true)            6368           6511         122          3.3         303.7       1.7X
    ```
    
    3. manual test on TPC-DS
    TPC-DS data size: 2TB.
    This improvement is valid for TPC-DS q67 and no regression for other test cases.
    
    | TPC-DS Query   | Before(Seconds)  | After(Seconds)  | Speedup(Percent)  |
    |  ----  | ----  | ----  | ----  |
    | q44 | 29.945 | 28.862 | 3.75% |
    | q67 | 997.7585 | 412.046 | 142.15% |
    | q70 | 20.149 | 19.955 | 0.97% |
    | All TPC-DS | 7076.6715 | 6472.0535 | 9.34% |
    
    There is an extreme case for this optimization. If the window group size is small enough, we still sort by partition key, order by key before shuffle. The worst case scenario is that there is only one row of data in each window group. I changed the partition of the window frame in q67 of TPC-DS from i_ The category is replaced by the store_ Primary key ss of sales table_ ticket_ Number, you can test this worst-case scenario.
    
    | TPC-DS Query   | Before(Seconds)  | After(Seconds)  | Speedup(Percent)  |
    |  ----  | ----  | ----  | ----  |
    | q67-changed | 2655.285 | 3062.245 | -13.29% |
    
    Closes #38799 from beliefer/SPARK-37099_new.
    
    Authored-by: Jiaan Geng <be...@163.com>
    Signed-off-by: Wenchen Fan <we...@databricks.com>
---
 .../apache/spark/sql/catalyst/dsl/package.scala    |   7 +
 .../optimizer/InsertWindowGroupLimit.scala         | 102 +++++++
 .../plans/logical/basicLogicalOperators.scala      |  16 ++
 .../sql/catalyst/rules/RuleIdCollection.scala      |   1 +
 .../spark/sql/catalyst/trees/TreePatterns.scala    |   1 +
 .../org/apache/spark/sql/internal/SQLConf.scala    |  13 +
 .../optimizer/InsertWindowGroupLimitSuite.scala    | 287 ++++++++++++++++++++
 sql/core/benchmarks/TopKBenchmark-results.txt      |  22 ++
 .../spark/sql/execution/QueryExecution.scala       |   6 +-
 .../RemoveRedundantWindowGroupLimits.scala         |  37 +++
 .../spark/sql/execution/SparkOptimizer.scala       |   1 +
 .../apache/spark/sql/execution/SparkPlanner.scala  |   1 +
 .../spark/sql/execution/SparkStrategies.scala      |  12 +
 .../execution/adaptive/AdaptiveSparkPlanExec.scala |   1 +
 .../execution/window/WindowGroupLimitExec.scala    | 252 ++++++++++++++++++
 .../approved-plans-v1_4/q44.sf100/explain.txt      | 292 +++++++++++---------
 .../approved-plans-v1_4/q44.sf100/simplified.txt   |  80 +++---
 .../approved-plans-v1_4/q44/explain.txt            | 296 ++++++++++++---------
 .../approved-plans-v1_4/q44/simplified.txt         |  80 +++---
 .../approved-plans-v1_4/q67.sf100/explain.txt      | 107 ++++----
 .../approved-plans-v1_4/q67.sf100/simplified.txt   | 105 ++++----
 .../approved-plans-v1_4/q67/explain.txt            | 101 ++++---
 .../approved-plans-v1_4/q67/simplified.txt         |  89 ++++---
 .../approved-plans-v1_4/q70.sf100/explain.txt      | 129 ++++-----
 .../approved-plans-v1_4/q70.sf100/simplified.txt   |  47 ++--
 .../approved-plans-v1_4/q70/explain.txt            | 129 ++++-----
 .../approved-plans-v1_4/q70/simplified.txt         |  51 ++--
 .../approved-plans-v2_7/q67a.sf100/explain.txt     | 187 +++++++------
 .../approved-plans-v2_7/q67a.sf100/simplified.txt  | 237 +++++++++--------
 .../approved-plans-v2_7/q67a/explain.txt           | 181 +++++++------
 .../approved-plans-v2_7/q67a/simplified.txt        | 221 +++++++--------
 .../approved-plans-v2_7/q70a.sf100/explain.txt     | 181 +++++++------
 .../approved-plans-v2_7/q70a.sf100/simplified.txt  |  47 ++--
 .../approved-plans-v2_7/q70a/explain.txt           | 181 +++++++------
 .../approved-plans-v2_7/q70a/simplified.txt        |  51 ++--
 .../spark/sql/DataFrameWindowFunctionsSuite.scala  | 164 ++++++++++++
 .../RemoveRedundantWindowGroupLimitsSuite.scala    |  79 ++++++
 .../sql/execution/benchmark/TopKBenchmark.scala    |  76 ++++++
 38 files changed, 2584 insertions(+), 1286 deletions(-)

diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala
index ecd1ed94ffd..c6cc863108d 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala
@@ -467,6 +467,13 @@ package object dsl {
           orderSpec: Seq[SortOrder]): LogicalPlan =
         Window(windowExpressions, partitionSpec, orderSpec, logicalPlan)
 
+      def windowGroupLimit(
+          partitionSpec: Seq[Expression],
+          orderSpec: Seq[SortOrder],
+          rankLikeFunction: Expression,
+          limit: Int): LogicalPlan =
+        WindowGroupLimit(partitionSpec, orderSpec, rankLikeFunction, limit, logicalPlan)
+
       // TODO: Remove at Spark 4.0.0
       @deprecated("Use subquery(alias: String)", "3.4.0")
       def subquery(alias: Symbol): LogicalPlan = SubqueryAlias(alias.name, logicalPlan)
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InsertWindowGroupLimit.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InsertWindowGroupLimit.scala
new file mode 100644
index 00000000000..bc767de6862
--- /dev/null
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InsertWindowGroupLimit.scala
@@ -0,0 +1,102 @@
+/*
+ * 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.catalyst.optimizer
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, CurrentRow, DenseRank, EqualTo, Expression, GreaterThan, GreaterThanOrEqual, IntegerLiteral, LessThan, LessThanOrEqual, NamedExpression, PredicateHelper, Rank, RowFrame, RowNumber, SpecifiedWindowFrame, UnboundedPreceding, WindowExpression, WindowSpecDefinition}
+import org.apache.spark.sql.catalyst.plans.logical.{Filter, LocalRelation, LogicalPlan, Window, WindowGroupLimit}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{FILTER, WINDOW}
+
+/**
+ * Inserts a `WindowGroupLimit` below `Window` if the `Window` has rank-like functions
+ * and the function results are further filtered by limit-like predicates. Example query:
+ * {{{
+ *   SELECT *, ROW_NUMBER() OVER(PARTITION BY k ORDER BY a) AS rn FROM Tab1 WHERE rn = 5
+ *   SELECT *, ROW_NUMBER() OVER(PARTITION BY k ORDER BY a) AS rn FROM Tab1 WHERE 5 = rn
+ *   SELECT *, ROW_NUMBER() OVER(PARTITION BY k ORDER BY a) AS rn FROM Tab1 WHERE rn < 5
+ *   SELECT *, ROW_NUMBER() OVER(PARTITION BY k ORDER BY a) AS rn FROM Tab1 WHERE 5 > rn
+ *   SELECT *, ROW_NUMBER() OVER(PARTITION BY k ORDER BY a) AS rn FROM Tab1 WHERE rn <= 5
+ *   SELECT *, ROW_NUMBER() OVER(PARTITION BY k ORDER BY a) AS rn FROM Tab1 WHERE 5 >= rn
+ * }}}
+ */
+object InsertWindowGroupLimit extends Rule[LogicalPlan] with PredicateHelper {
+
+  /**
+   * Extract all the limit values from predicates.
+   */
+  def extractLimits(condition: Expression, attr: Attribute): Option[Int] = {
+    val limits = splitConjunctivePredicates(condition).collect {
+      case EqualTo(IntegerLiteral(limit), e) if e.semanticEquals(attr) => limit
+      case EqualTo(e, IntegerLiteral(limit)) if e.semanticEquals(attr) => limit
+      case LessThan(e, IntegerLiteral(limit)) if e.semanticEquals(attr) => limit - 1
+      case GreaterThan(IntegerLiteral(limit), e) if e.semanticEquals(attr) => limit - 1
+      case LessThanOrEqual(e, IntegerLiteral(limit)) if e.semanticEquals(attr) => limit
+      case GreaterThanOrEqual(IntegerLiteral(limit), e) if e.semanticEquals(attr) => limit
+    }
+
+    if (limits.nonEmpty) Some(limits.min) else None
+  }
+
+  private def support(
+      windowExpression: NamedExpression): Boolean = windowExpression match {
+    case Alias(WindowExpression(_: Rank | _: DenseRank | _: RowNumber, WindowSpecDefinition(_, _,
+    SpecifiedWindowFrame(RowFrame, UnboundedPreceding, CurrentRow))), _) => true
+    case _ => false
+  }
+
+  def apply(plan: LogicalPlan): LogicalPlan = {
+    if (conf.windowGroupLimitThreshold == -1) return plan
+
+    plan.transformWithPruning(_.containsAllPatterns(FILTER, WINDOW), ruleId) {
+      case filter @ Filter(condition,
+        window @ Window(windowExpressions, partitionSpec, orderSpec, child))
+        if !child.isInstanceOf[WindowGroupLimit] && windowExpressions.exists(support) &&
+          orderSpec.nonEmpty =>
+        val limits = windowExpressions.collect {
+          case alias @ Alias(WindowExpression(rankLikeFunction, _), _) if support(alias) =>
+            extractLimits(condition, alias.toAttribute).map((_, rankLikeFunction))
+        }.flatten
+
+        if (limits.isEmpty) {
+          filter
+        } else {
+          val (rowNumberLimits, otherLimits) = limits.partition(_._2.isInstanceOf[RowNumber])
+          // Pick RowNumber first as it's cheaper to evaluate.
+          val selectedLimits = if (rowNumberLimits.isEmpty) {
+            otherLimits
+          } else {
+            rowNumberLimits
+          }
+          // Pick a rank-like function with the smallest limit
+          selectedLimits.minBy(_._1) match {
+            case (limit, rankLikeFunction) if limit <= conf.windowGroupLimitThreshold =>
+              if (limit > 0) {
+                val windowGroupLimit =
+                  WindowGroupLimit(partitionSpec, orderSpec, rankLikeFunction, limit, child)
+                val newWindow = window.withNewChildren(Seq(windowGroupLimit))
+                filter.withNewChildren(Seq(newWindow))
+              } else {
+                LocalRelation(filter.output, data = Seq.empty, isStreaming = filter.isStreaming)
+              }
+            case _ =>
+              filter
+          }
+        }
+    }
+  }
+}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala
index 74929bf5d79..79374d11802 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala
@@ -1230,6 +1230,22 @@ case class Window(
     copy(child = newChild)
 }
 
+case class WindowGroupLimit(
+    partitionSpec: Seq[Expression],
+    orderSpec: Seq[SortOrder],
+    rankLikeFunction: Expression,
+    limit: Int,
+    child: LogicalPlan) extends UnaryNode {
+  assert(orderSpec.nonEmpty && limit > 0)
+
+  override def output: Seq[Attribute] = child.output
+  override def maxRows: Option[Long] = child.maxRows
+  override def maxRowsPerPartition: Option[Long] = child.maxRowsPerPartition
+  final override val nodePatterns: Seq[TreePattern] = Seq(WINDOW_GROUP_LIMIT)
+  override protected def withNewChildInternal(newChild: LogicalPlan): WindowGroupLimit =
+    copy(child = newChild)
+}
+
 object Expand {
   /**
    * Build bit mask from attributes of selected grouping set. A bit in the bitmask is corresponding
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleIdCollection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleIdCollection.scala
index 4be3f97dca8..21a9421244c 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleIdCollection.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleIdCollection.scala
@@ -120,6 +120,7 @@ object RuleIdCollection {
       "org.apache.spark.sql.catalyst.optimizer.EliminateMapObjects" ::
       "org.apache.spark.sql.catalyst.optimizer.EliminateOuterJoin" ::
       "org.apache.spark.sql.catalyst.optimizer.EliminateSerialization" ::
+      "org.apache.spark.sql.catalyst.optimizer.InsertWindowGroupLimit" ::
       "org.apache.spark.sql.catalyst.optimizer.LikeSimplification" ::
       "org.apache.spark.sql.catalyst.optimizer.LimitPushDown" ::
       "org.apache.spark.sql.catalyst.optimizer.LimitPushDownThroughWindow" ::
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreePatterns.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreePatterns.scala
index 48db1a4408d..11b47b7d5c8 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreePatterns.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreePatterns.scala
@@ -126,6 +126,7 @@ object TreePattern extends Enumeration  {
   val TEMP_RESOLVED_COLUMN: Value = Value
   val TYPED_FILTER: Value = Value
   val WINDOW: Value = Value
+  val WINDOW_GROUP_LIMIT: Value = Value
   val WITH_WINDOW_DEFINITION: Value = Value
 
   // Unresolved expression patterns (Alphabetically ordered)
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
index 32042ffd862..e7a673ae7b8 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
@@ -2628,6 +2628,17 @@ object SQLConf {
       .intConf
       .createWithDefault(SHUFFLE_SPILL_NUM_ELEMENTS_FORCE_SPILL_THRESHOLD.defaultValue.get)
 
+  val WINDOW_GROUP_LIMIT_THRESHOLD =
+    buildConf("spark.sql.optimizer.windowGroupLimitThreshold")
+      .internal()
+      .doc("Threshold for triggering `InsertWindowGroupLimit`. " +
+        "0 means the output results is empty. -1 means disabling the optimization.")
+      .version("3.5.0")
+      .intConf
+      .checkValue(_ >= -1,
+        "The threshold of window group limit must be -1, 0 or positive integer.")
+      .createWithDefault(1000)
+
   val SESSION_WINDOW_BUFFER_IN_MEMORY_THRESHOLD =
     buildConf("spark.sql.sessionWindow.buffer.in.memory.threshold")
       .internal()
@@ -4716,6 +4727,8 @@ class SQLConf extends Serializable with Logging {
 
   def windowExecBufferSpillThreshold: Int = getConf(WINDOW_EXEC_BUFFER_SPILL_THRESHOLD)
 
+  def windowGroupLimitThreshold: Int = getConf(WINDOW_GROUP_LIMIT_THRESHOLD)
+
   def sessionWindowBufferInMemoryThreshold: Int = getConf(SESSION_WINDOW_BUFFER_IN_MEMORY_THRESHOLD)
 
   def sessionWindowBufferSpillThreshold: Int = getConf(SESSION_WINDOW_BUFFER_SPILL_THRESHOLD)
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/InsertWindowGroupLimitSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/InsertWindowGroupLimitSuite.scala
new file mode 100644
index 00000000000..219a8ce8944
--- /dev/null
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/InsertWindowGroupLimitSuite.scala
@@ -0,0 +1,287 @@
+/*
+ * 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.catalyst.optimizer
+
+import org.apache.spark.sql.catalyst.dsl.expressions._
+import org.apache.spark.sql.catalyst.dsl.plans._
+import org.apache.spark.sql.catalyst.expressions.{CurrentRow, DenseRank, Literal, NthValue, NTile, Rank, RowFrame, RowNumber, SpecifiedWindowFrame, UnboundedPreceding}
+import org.apache.spark.sql.catalyst.plans.PlanTest
+import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan}
+import org.apache.spark.sql.catalyst.rules.RuleExecutor
+import org.apache.spark.sql.internal.SQLConf
+
+class InsertWindowGroupLimitSuite extends PlanTest {
+  private object Optimize extends RuleExecutor[LogicalPlan] {
+    val batches =
+      Batch("Insert WindowGroupLimit", FixedPoint(10),
+        CollapseProject,
+        RemoveNoopOperators,
+        PushDownPredicates,
+        InsertWindowGroupLimit) :: Nil
+  }
+
+  private object WithoutOptimize extends RuleExecutor[LogicalPlan] {
+    val batches =
+      Batch("Insert WindowGroupLimit", FixedPoint(10),
+        CollapseProject,
+        RemoveNoopOperators,
+        PushDownPredicates) :: Nil
+  }
+
+  private val testRelation = LocalRelation(
+    Seq("a".attr.int, "b".attr.int, "c".attr.int))
+  private val a = testRelation.output(0)
+  private val b = testRelation.output(1)
+  private val c = testRelation.output(2)
+  private val rankLikeFunctions = Seq(RowNumber(), Rank(c :: Nil), DenseRank(c :: Nil))
+  private val unsupportedFunctions = Seq(new NthValue(c, Literal(1)), new NTile())
+  private val windowFrame = SpecifiedWindowFrame(RowFrame, UnboundedPreceding, CurrentRow)
+  private val supportedConditions = Seq($"rn" === 2, $"rn" < 3, $"rn" <= 2)
+  private val unsupportedConditions = Seq($"rn" > 2, $"rn" === 1 || b < 2)
+
+  test("window without filter") {
+    for (function <- rankLikeFunctions) {
+      val originalQuery =
+        testRelation
+          .select(a, b, c,
+            windowExpr(function,
+              windowSpec(a :: Nil, c.desc :: Nil, windowFrame)).as("rn"))
+
+      comparePlans(
+        Optimize.execute(originalQuery.analyze),
+        WithoutOptimize.execute(originalQuery.analyze))
+    }
+  }
+
+  test("spark.sql.window.group.limit.threshold = -1") {
+    withSQLConf(SQLConf.WINDOW_GROUP_LIMIT_THRESHOLD.key -> "-1") {
+      for (condition <- supportedConditions; function <- rankLikeFunctions) {
+        val originalQuery =
+          testRelation
+            .select(a, b, c,
+              windowExpr(function,
+                windowSpec(a :: Nil, c.desc :: Nil, windowFrame)).as("rn"))
+            .where(condition)
+
+        comparePlans(
+          Optimize.execute(originalQuery.analyze),
+          WithoutOptimize.execute(originalQuery.analyze))
+      }
+    }
+  }
+
+  test("Insert window group limit node for top-k computation") {
+    for (condition <- supportedConditions; function <- rankLikeFunctions;
+      moreCond <- Seq(true, false)) {
+      val cond = if (moreCond) {
+        condition && b > 0
+      } else {
+        condition
+      }
+
+      val originalQuery0 =
+        testRelation
+          .select(a, b, c,
+            windowExpr(function,
+              windowSpec(a :: Nil, c.desc :: Nil, windowFrame)).as("rn"))
+          .where(cond)
+
+      val correctAnswer0 =
+        testRelation
+          .windowGroupLimit(a :: Nil, c.desc :: Nil, function, 2)
+          .select(a, b, c,
+            windowExpr(function,
+              windowSpec(a :: Nil, c.desc :: Nil, windowFrame)).as("rn"))
+          .where(cond)
+
+      comparePlans(
+        Optimize.execute(originalQuery0.analyze),
+        WithoutOptimize.execute(correctAnswer0.analyze))
+
+      val originalQuery1 =
+        testRelation
+          .select(a, b, c,
+            windowExpr(function,
+              windowSpec(Nil, c.desc :: Nil, windowFrame)).as("rn"))
+          .where(cond)
+
+      val correctAnswer1 =
+        testRelation
+          .windowGroupLimit(Nil, c.desc :: Nil, function, 2)
+          .select(a, b, c,
+            windowExpr(function,
+              windowSpec(Nil, c.desc :: Nil, windowFrame)).as("rn"))
+          .where(cond)
+
+      comparePlans(
+        Optimize.execute(originalQuery1.analyze),
+        WithoutOptimize.execute(correctAnswer1.analyze))
+    }
+  }
+
+  test("Unsupported conditions") {
+    for (condition <- unsupportedConditions; function <- rankLikeFunctions) {
+      val originalQuery =
+        testRelation
+          .select(a, b, c,
+            windowExpr(function,
+              windowSpec(a :: Nil, c.desc :: Nil, windowFrame)).as("rn"))
+          .where(condition)
+
+      comparePlans(
+        Optimize.execute(originalQuery.analyze),
+        WithoutOptimize.execute(originalQuery.analyze))
+    }
+  }
+
+  test("Unsupported window functions") {
+    for (condition <- supportedConditions; function <- unsupportedFunctions) {
+      val originalQuery =
+        testRelation
+          .select(a, b, c,
+            windowExpr(function,
+              windowSpec(a :: Nil, c.desc :: Nil, windowFrame)).as("rn"))
+          .where(condition)
+
+      comparePlans(
+        Optimize.execute(originalQuery.analyze),
+        WithoutOptimize.execute(originalQuery.analyze))
+    }
+  }
+
+  test("Insert window group limit node for top-k computation: multiple rank-like functions") {
+    rankLikeFunctions.foreach { function =>
+      val originalQuery =
+        testRelation
+          .select(a, b, c,
+            windowExpr(function,
+              windowSpec(a :: Nil, c.desc :: Nil, windowFrame)).as("rn"),
+            windowExpr(function,
+              windowSpec(a :: Nil, c.desc :: Nil, windowFrame)).as("rn2"))
+          .where('rn < 2 && 'rn2 === 3)
+
+      val correctAnswer =
+        testRelation
+          .windowGroupLimit(a :: Nil, c.desc :: Nil, function, 1)
+          .select(a, b, c,
+            windowExpr(function,
+              windowSpec(a :: Nil, c.desc :: Nil, windowFrame)).as("rn"),
+            windowExpr(function,
+              windowSpec(a :: Nil, c.desc :: Nil, windowFrame)).as("rn2"))
+          .where('rn < 2 && 'rn2 === 3)
+
+      comparePlans(
+        Optimize.execute(originalQuery.analyze),
+        WithoutOptimize.execute(correctAnswer.analyze))
+    }
+  }
+
+  test("multiple different rank-like window function and only one used in filter") {
+    val originalQuery =
+      testRelation
+        .select(a, b, c,
+          windowExpr(RowNumber(),
+            windowSpec(a :: Nil, c.desc :: Nil, windowFrame)).as("rn"),
+          windowExpr(Rank(c :: Nil),
+            windowSpec(a :: Nil, c.desc :: Nil, windowFrame)).as("rank"))
+        .where('rn < 2)
+
+    val correctAnswer =
+      testRelation
+        .windowGroupLimit(a :: Nil, c.desc :: Nil, RowNumber(), 1)
+        .select(a, b, c,
+          windowExpr(RowNumber(),
+            windowSpec(a :: Nil, c.desc :: Nil, windowFrame)).as("rn"),
+          windowExpr(Rank(c :: Nil),
+            windowSpec(a :: Nil, c.desc :: Nil, windowFrame)).as("rank"))
+        .where('rn < 2)
+
+    comparePlans(
+      Optimize.execute(originalQuery.analyze),
+      WithoutOptimize.execute(correctAnswer.analyze))
+  }
+
+  test("rank-like window function with unsupported window function") {
+    val originalQuery =
+      testRelation
+        .select(a, b, c,
+          windowExpr(RowNumber(),
+            windowSpec(a :: Nil, c.desc :: Nil, windowFrame)).as("rn"),
+          windowExpr(new NthValue(c, Literal(1)),
+            windowSpec(a :: Nil, c.desc :: Nil, windowFrame)).as("rank"))
+        .where('rn < 2)
+
+    val correctAnswer =
+      testRelation
+        .windowGroupLimit(a :: Nil, c.desc :: Nil, RowNumber(), 1)
+        .select(a, b, c,
+          windowExpr(RowNumber(),
+            windowSpec(a :: Nil, c.desc :: Nil, windowFrame)).as("rn"),
+          windowExpr(new NthValue(c, Literal(1)),
+            windowSpec(a :: Nil, c.desc :: Nil, windowFrame)).as("rank"))
+        .where('rn < 2)
+
+    comparePlans(
+      Optimize.execute(originalQuery.analyze),
+      WithoutOptimize.execute(correctAnswer.analyze))
+  }
+
+  test("multiple different rank-like window function with filter") {
+    val originalQuery =
+      testRelation
+        .select(a, b, c,
+          windowExpr(RowNumber(),
+            windowSpec(a :: Nil, c.desc :: Nil, windowFrame)).as("rn"),
+          windowExpr(Rank(c :: Nil),
+            windowSpec(a :: Nil, c.desc :: Nil, windowFrame)).as("rank"))
+        .where('rn < 2 && 'rank === 3)
+
+    val correctAnswer =
+      testRelation
+        .windowGroupLimit(a :: Nil, c.desc :: Nil, RowNumber(), 1)
+        .select(a, b, c,
+          windowExpr(RowNumber(),
+            windowSpec(a :: Nil, c.desc :: Nil, windowFrame)).as("rn"),
+          windowExpr(Rank(c :: Nil),
+            windowSpec(a :: Nil, c.desc :: Nil, windowFrame)).as("rank"))
+        .where('rn < 2 && 'rank === 3)
+
+    comparePlans(
+      Optimize.execute(originalQuery.analyze),
+      WithoutOptimize.execute(correctAnswer.analyze))
+  }
+
+  test("Insert window group limit node for top-k computation: empty relation") {
+    Seq($"rn" === 0, $"rn" < 1, $"rn" <= 0).foreach { condition =>
+      rankLikeFunctions.foreach { function =>
+        val originalQuery =
+          testRelation
+            .select(a, b, c,
+              windowExpr(function,
+                windowSpec(a :: Nil, c.desc :: Nil, windowFrame)).as("rn"))
+            .where(condition)
+
+        val correctAnswer = LocalRelation(originalQuery.output)
+
+        comparePlans(
+          Optimize.execute(originalQuery.analyze),
+          WithoutOptimize.execute(correctAnswer.analyze))
+      }
+    }
+  }
+}
diff --git a/sql/core/benchmarks/TopKBenchmark-results.txt b/sql/core/benchmarks/TopKBenchmark-results.txt
new file mode 100644
index 00000000000..146f1479a63
--- /dev/null
+++ b/sql/core/benchmarks/TopKBenchmark-results.txt
@@ -0,0 +1,22 @@
+================================================================================================
+Top-K Computation
+================================================================================================
+
+Java HotSpot(TM) 64-Bit Server VM 1.8.0_311-b11 on Mac OS X 10.16
+Intel(R) Core(TM) i7-9750H CPU @ 2.60GHz
+Benchmark Top-K:                                                 Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
+-----------------------------------------------------------------------------------------------------------------------------------------------
+ROW_NUMBER (PARTITION: , WindowGroupLimit: false)                        10711          11068         363          2.0         510.8       1.0X
+ROW_NUMBER (PARTITION: , WindowGroupLimit: true)                          2611           2929         210          8.0         124.5       4.1X
+ROW_NUMBER (PARTITION: PARTITION BY b, WindowGroupLimit: false)          23829          24085         178          0.9        1136.2       0.4X
+ROW_NUMBER (PARTITION: PARTITION BY b, WindowGroupLimit: true)            6381           6515          97          3.3         304.3       1.7X
+RANK (PARTITION: , WindowGroupLimit: false)                              11459          11798         223          1.8         546.4       0.9X
+RANK (PARTITION: , WindowGroupLimit: true)                                2588           2837         162          8.1         123.4       4.1X
+RANK (PARTITION: PARTITION BY b, WindowGroupLimit: false)                24560          24707         108          0.9        1171.1       0.4X
+RANK (PARTITION: PARTITION BY b, WindowGroupLimit: true)                  6395           6530         109          3.3         304.9       1.7X
+DENSE_RANK (PARTITION: , WindowGroupLimit: false)                        11563          11740         124          1.8         551.3       0.9X
+DENSE_RANK (PARTITION: , WindowGroupLimit: true)                          2563           2819         168          8.2         122.2       4.2X
+DENSE_RANK (PARTITION: PARTITION BY b, WindowGroupLimit: false)          24482          24578          64          0.9        1167.4       0.4X
+DENSE_RANK (PARTITION: PARTITION BY b, WindowGroupLimit: true)            6368           6511         122          3.3         303.7       1.7X
+
+
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala
index 362615770a3..ea713e390e0 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala
@@ -431,9 +431,11 @@ object QueryExecution {
       // `ReplaceHashWithSortAgg` needs to be added after `EnsureRequirements` to guarantee the
       // sort order of each node is checked to be valid.
       ReplaceHashWithSortAgg,
-      // `RemoveRedundantSorts` needs to be added after `EnsureRequirements` to guarantee the same
-      // number of partitions when instantiating PartitioningCollection.
+      // `RemoveRedundantSorts` and `RemoveRedundantWindowGroupLimits` needs to be added after
+      // `EnsureRequirements` to guarantee the same number of partitions when instantiating
+      // PartitioningCollection.
       RemoveRedundantSorts,
+      RemoveRedundantWindowGroupLimits,
       DisableUnnecessaryBucketedScan,
       ApplyColumnarRulesAndInsertTransitions(
         sparkSession.sessionState.columnarRules, outputsColumnar = false),
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/RemoveRedundantWindowGroupLimits.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/RemoveRedundantWindowGroupLimits.scala
new file mode 100644
index 00000000000..68b460f64e4
--- /dev/null
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/RemoveRedundantWindowGroupLimits.scala
@@ -0,0 +1,37 @@
+/*
+ * 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
+
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.execution.window.{Final, Partial, WindowGroupLimitExec}
+
+/**
+ * Remove redundant partial WindowGroupLimitExec node from the spark plan. A partial
+ * WindowGroupLimitExec node is redundant when its child satisfies its required child distribution.
+ */
+object RemoveRedundantWindowGroupLimits extends Rule[SparkPlan] {
+
+  def apply(plan: SparkPlan): SparkPlan = plan transform {
+    case outer @ WindowGroupLimitExec(
+    _, _, _, _, Final, WindowGroupLimitExec(_, _, _, _, Partial, child))
+      if child.outputPartitioning.satisfies(outer.requiredChildDistribution.head) =>
+      val newOuter = outer.withNewChildren(Seq(child))
+      newOuter
+  }
+
+}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala
index 8c420838ca2..5ecfe5c32dd 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala
@@ -84,6 +84,7 @@ class SparkOptimizer(
       PushPredicateThroughNonJoin,
       PushProjectionThroughLimit,
       RemoveNoopOperators) :+
+    Batch("Insert window group limit", Once, InsertWindowGroupLimit) :+
     Batch("User Provided Optimizers", fixedPoint, experimentalMethods.extraOptimizations: _*) :+
     Batch("Replace CTE with Repartition", Once, ReplaceCTERefWithRepartition)
 
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanner.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanner.scala
index 6994aaf47df..6f7dd852cfe 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanner.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanner.scala
@@ -41,6 +41,7 @@ class SparkPlanner(val session: SparkSession, val experimentalMethods: Experimen
       SpecialLimits ::
       Aggregation ::
       Window ::
+      WindowGroupLimit ::
       JoinSelection ::
       InMemoryScans ::
       SparkScripts ::
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala
index cd4485e3822..47b3fa2c684 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala
@@ -628,6 +628,18 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] {
     }
   }
 
+  object WindowGroupLimit extends Strategy {
+    def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match {
+      case logical.WindowGroupLimit(partitionSpec, orderSpec, rankLikeFunction, limit, child) =>
+        val partialWindowGroupLimit = execution.window.WindowGroupLimitExec(partitionSpec,
+          orderSpec, rankLikeFunction, limit, execution.window.Partial, planLater(child))
+        val finalWindowGroupLimit = execution.window.WindowGroupLimitExec(partitionSpec, orderSpec,
+          rankLikeFunction, limit, execution.window.Final, partialWindowGroupLimit)
+        finalWindowGroupLimit :: Nil
+      case _ => Nil
+    }
+  }
+
   protected lazy val singleRowRdd = session.sparkContext.parallelize(Seq(InternalRow()), 1)
 
   object InMemoryScans extends Strategy {
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala
index 395e5468b64..22b653c281f 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala
@@ -122,6 +122,7 @@ case class AdaptiveSparkPlanExec(
       ValidateSparkPlan,
       ReplaceHashWithSortAgg,
       RemoveRedundantSorts,
+      RemoveRedundantWindowGroupLimits,
       DisableUnnecessaryBucketedScan,
       OptimizeSkewedJoin(ensureRequirements)
     ) ++ context.session.sessionState.adaptiveRulesHolder.queryStagePrepRules
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowGroupLimitExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowGroupLimitExec.scala
new file mode 100644
index 00000000000..2aee094e276
--- /dev/null
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowGroupLimitExec.scala
@@ -0,0 +1,252 @@
+/*
+ * 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.
+ * @param limit The limit for rank value.
+ * @param mode The mode describes [[WindowGroupLimitExec]] before or after shuffle.
+ * @param child The child spark plan.
+ */
+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 if partitionSpec.isEmpty =>
+      child.execute().mapPartitionsInternal(SimpleLimitIterator(_, limit))
+    case _: RowNumber =>
+      child.execute().mapPartitionsInternal(new GroupedLimitIterator(_, output, partitionSpec,
+        (input: Iterator[InternalRow]) => SimpleLimitIterator(input, limit)))
+    case _: Rank if partitionSpec.isEmpty =>
+      child.execute().mapPartitionsInternal(RankLimitIterator(output, _, orderSpec, limit))
+    case _: Rank =>
+      child.execute().mapPartitionsInternal(new GroupedLimitIterator(_, output, partitionSpec,
+        (input: Iterator[InternalRow]) => RankLimitIterator(output, input, orderSpec, limit)))
+    case _: DenseRank if partitionSpec.isEmpty =>
+   child.execute().mapPartitionsInternal(DenseRankLimitIterator(output, _, orderSpec, limit))
+    case _: DenseRank =>
+      child.execute().mapPartitionsInternal(new GroupedLimitIterator(_, output, partitionSpec,
+        (input: Iterator[InternalRow]) => DenseRankLimitIterator(output, input, orderSpec, limit)))
+  }
+
+  override protected def withNewChildInternal(newChild: SparkPlan): WindowGroupLimitExec =
+    copy(child = newChild)
+}
+
+abstract class BaseLimitIterator extends Iterator[InternalRow] {
+
+  def input: Iterator[InternalRow]
+
+  def limit: Int
+
+  var rank = 0
+
+  var nextRow: UnsafeRow = null
+
+  // Increase the rank value.
+  def increaseRank(): Unit
+
+  override def hasNext: Boolean = rank < limit && input.hasNext
+
+  override def next(): InternalRow = {
+    if (!hasNext) throw new NoSuchElementException
+    nextRow = input.next().asInstanceOf[UnsafeRow]
+    increaseRank()
+    nextRow
+  }
+
+  def reset(): Unit
+}
+
+case class SimpleLimitIterator(
+    input: Iterator[InternalRow],
+    limit: Int) extends BaseLimitIterator {
+
+  override def increaseRank(): Unit = {
+    rank += 1
+  }
+
+  override def reset(): Unit = {
+    rank = 0
+  }
+}
+
+trait OrderSpecProvider {
+  def output: Seq[Attribute]
+  def orderSpec: Seq[SortOrder]
+
+  val ordering = GenerateOrdering.generate(orderSpec, output)
+  var currentRankRow: UnsafeRow = null
+}
+
+case class RankLimitIterator(
+    output: Seq[Attribute],
+    input: Iterator[InternalRow],
+    orderSpec: Seq[SortOrder],
+    limit: Int) extends BaseLimitIterator with OrderSpecProvider {
+
+  var count = 0
+
+  override def increaseRank(): Unit = {
+    if (count == 0) {
+      currentRankRow = nextRow.copy()
+    } else {
+      if (ordering.compare(currentRankRow, nextRow) != 0) {
+        rank = count
+        currentRankRow = nextRow.copy()
+      }
+    }
+    count += 1
+  }
+
+  override def reset(): Unit = {
+    rank = 0
+    count = 0
+  }
+}
+
+case class DenseRankLimitIterator(
+    output: Seq[Attribute],
+    input: Iterator[InternalRow],
+    orderSpec: Seq[SortOrder],
+    limit: Int) extends BaseLimitIterator with OrderSpecProvider {
+
+  override def increaseRank(): Unit = {
+    if (currentRankRow == null) {
+      currentRankRow = nextRow.copy()
+    } else {
+      if (ordering.compare(currentRankRow, nextRow) != 0) {
+        rank += 1
+        currentRankRow = nextRow.copy()
+      }
+    }
+  }
+
+  override def reset(): Unit = {
+    rank = 0
+  }
+}
+
+class GroupedLimitIterator(
+    input: Iterator[InternalRow],
+    output: Seq[Attribute],
+    partitionSpec: Seq[Expression],
+    createLimitIterator: Iterator[InternalRow] => BaseLimitIterator)
+  extends 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 = input.hasNext
+    if (nextRowAvailable) {
+      nextRow = input.next().asInstanceOf[UnsafeRow]
+      nextGroup = grouping(nextRow)
+    }
+  }
+  fetchNextRow()
+
+  var groupIterator: GroupIterator = _
+  var limitIterator: BaseLimitIterator = _
+  if (nextRowAvailable) {
+    groupIterator = new GroupIterator()
+    limitIterator = createLimitIterator(groupIterator)
+  }
+
+  override final def hasNext: Boolean = nextRowAvailable && {
+    if (!limitIterator.hasNext) {
+      // if `limitIterator.hasNext` is false, we should jump to the next group if present
+      groupIterator.skipRemainingRows()
+      limitIterator.reset()
+    }
+    limitIterator.hasNext
+  }
+
+  override final def next(): InternalRow = {
+    if (!hasNext) throw new NoSuchElementException
+    limitIterator.next()
+  }
+
+  class GroupIterator() extends Iterator[InternalRow] {
+    // Before we start to fetch new input rows, make a copy of nextGroup.
+    var currentGroup = nextGroup.copy()
+
+    def hasNext: Boolean = nextRowAvailable && nextGroup == currentGroup
+
+    def next(): InternalRow = {
+      if (!hasNext) throw new NoSuchElementException
+      val currentRow = nextRow.copy()
+      fetchNextRow()
+      currentRow
+    }
+
+    def skipRemainingRows(): Unit = {
+      // Skip all the remaining rows in this group
+      while (hasNext) {
+        fetchNextRow()
+      }
+
+      // Switch to next group
+      if (nextRowAvailable) currentGroup = nextGroup.copy()
+    }
+  }
+}
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.sf100/explain.txt
index 0fba1f74394..4e21aba538b 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.sf100/explain.txt
@@ -1,35 +1,44 @@
 == Physical Plan ==
-TakeOrderedAndProject (31)
-+- * Project (30)
-   +- * BroadcastHashJoin Inner BuildRight (29)
-      :- * Project (27)
-      :  +- * BroadcastHashJoin Inner BuildRight (26)
-      :     :- * Project (21)
-      :     :  +- * BroadcastHashJoin Inner BuildRight (20)
-      :     :     :- * Project (13)
-      :     :     :  +- * Filter (12)
-      :     :     :     +- Window (11)
-      :     :     :        +- * Sort (10)
-      :     :     :           +- Exchange (9)
-      :     :     :              +- * Filter (8)
-      :     :     :                 +- * HashAggregate (7)
-      :     :     :                    +- Exchange (6)
-      :     :     :                       +- * HashAggregate (5)
-      :     :     :                          +- * Project (4)
-      :     :     :                             +- * Filter (3)
-      :     :     :                                +- * ColumnarToRow (2)
-      :     :     :                                   +- Scan parquet spark_catalog.default.store_sales (1)
-      :     :     +- BroadcastExchange (19)
-      :     :        +- * Project (18)
-      :     :           +- * Filter (17)
-      :     :              +- Window (16)
-      :     :                 +- * Sort (15)
-      :     :                    +- ReusedExchange (14)
-      :     +- BroadcastExchange (25)
-      :        +- * Filter (24)
-      :           +- * ColumnarToRow (23)
-      :              +- Scan parquet spark_catalog.default.item (22)
-      +- ReusedExchange (28)
+TakeOrderedAndProject (40)
++- * Project (39)
+   +- * BroadcastHashJoin Inner BuildRight (38)
+      :- * Project (36)
+      :  +- * BroadcastHashJoin Inner BuildRight (35)
+      :     :- * Project (30)
+      :     :  +- * BroadcastHashJoin Inner BuildRight (29)
+      :     :     :- * Project (16)
+      :     :     :  +- * Filter (15)
+      :     :     :     +- Window (14)
+      :     :     :        +- WindowGroupLimit (13)
+      :     :     :           +- * Sort (12)
+      :     :     :              +- Exchange (11)
+      :     :     :                 +- WindowGroupLimit (10)
+      :     :     :                    +- * Sort (9)
+      :     :     :                       +- * Filter (8)
+      :     :     :                          +- * HashAggregate (7)
+      :     :     :                             +- Exchange (6)
+      :     :     :                                +- * HashAggregate (5)
+      :     :     :                                   +- * Project (4)
+      :     :     :                                      +- * Filter (3)
+      :     :     :                                         +- * ColumnarToRow (2)
+      :     :     :                                            +- Scan parquet spark_catalog.default.store_sales (1)
+      :     :     +- BroadcastExchange (28)
+      :     :        +- * Project (27)
+      :     :           +- * Filter (26)
+      :     :              +- Window (25)
+      :     :                 +- WindowGroupLimit (24)
+      :     :                    +- * Sort (23)
+      :     :                       +- Exchange (22)
+      :     :                          +- WindowGroupLimit (21)
+      :     :                             +- * Sort (20)
+      :     :                                +- * Filter (19)
+      :     :                                   +- * HashAggregate (18)
+      :     :                                      +- ReusedExchange (17)
+      :     +- BroadcastExchange (34)
+      :        +- * Filter (33)
+      :           +- * ColumnarToRow (32)
+      :              +- Scan parquet spark_catalog.default.item (31)
+      +- ReusedExchange (37)
 
 
 (1) Scan parquet spark_catalog.default.store_sales
@@ -72,150 +81,191 @@ Results [2]: [ss_item_sk#1 AS item_sk#10, cast((avg(UnscaledValue(ss_net_profit#
 Input [2]: [item_sk#10, rank_col#11]
 Condition : (isnotnull(rank_col#11) AND (cast(rank_col#11 as decimal(13,7)) > (0.9 * Subquery scalar-subquery#12, [id=#13])))
 
-(9) Exchange
+(9) Sort [codegen id : 2]
+Input [2]: [item_sk#10, rank_col#11]
+Arguments: [rank_col#11 ASC NULLS FIRST], false, 0
+
+(10) WindowGroupLimit
+Input [2]: [item_sk#10, rank_col#11]
+Arguments: [rank_col#11 ASC NULLS FIRST], rank(rank_col#11), 10, Partial
+
+(11) Exchange
 Input [2]: [item_sk#10, rank_col#11]
 Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=2]
 
-(10) Sort [codegen id : 3]
+(12) Sort [codegen id : 3]
 Input [2]: [item_sk#10, rank_col#11]
 Arguments: [rank_col#11 ASC NULLS FIRST], false, 0
 
-(11) Window
+(13) WindowGroupLimit
+Input [2]: [item_sk#10, rank_col#11]
+Arguments: [rank_col#11 ASC NULLS FIRST], rank(rank_col#11), 10, Final
+
+(14) Window
 Input [2]: [item_sk#10, rank_col#11]
 Arguments: [rank(rank_col#11) windowspecdefinition(rank_col#11 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#14], [rank_col#11 ASC NULLS FIRST]
 
-(12) Filter [codegen id : 10]
+(15) Filter [codegen id : 10]
 Input [3]: [item_sk#10, rank_col#11, rnk#14]
 Condition : ((rnk#14 < 11) AND isnotnull(item_sk#10))
 
-(13) Project [codegen id : 10]
+(16) Project [codegen id : 10]
 Output [2]: [item_sk#10, rnk#14]
 Input [3]: [item_sk#10, rank_col#11, rnk#14]
 
-(14) ReusedExchange [Reuses operator id: 9]
-Output [2]: [item_sk#15, rank_col#16]
+(17) ReusedExchange [Reuses operator id: 6]
+Output [3]: [ss_item_sk#15, sum#16, count#17]
+
+(18) HashAggregate [codegen id : 5]
+Input [3]: [ss_item_sk#15, sum#16, count#17]
+Keys [1]: [ss_item_sk#15]
+Functions [1]: [avg(UnscaledValue(ss_net_profit#18))]
+Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_profit#18))#19]
+Results [2]: [ss_item_sk#15 AS item_sk#20, cast((avg(UnscaledValue(ss_net_profit#18))#19 / 100.0) as decimal(11,6)) AS rank_col#21]
+
+(19) Filter [codegen id : 5]
+Input [2]: [item_sk#20, rank_col#21]
+Condition : (isnotnull(rank_col#21) AND (cast(rank_col#21 as decimal(13,7)) > (0.9 * ReusedSubquery Subquery scalar-subquery#12, [id=#13])))
+
+(20) Sort [codegen id : 5]
+Input [2]: [item_sk#20, rank_col#21]
+Arguments: [rank_col#21 DESC NULLS LAST], false, 0
+
+(21) WindowGroupLimit
+Input [2]: [item_sk#20, rank_col#21]
+Arguments: [rank_col#21 DESC NULLS LAST], rank(rank_col#21), 10, Partial
+
+(22) Exchange
+Input [2]: [item_sk#20, rank_col#21]
+Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=3]
 
-(15) Sort [codegen id : 6]
-Input [2]: [item_sk#15, rank_col#16]
-Arguments: [rank_col#16 DESC NULLS LAST], false, 0
+(23) Sort [codegen id : 6]
+Input [2]: [item_sk#20, rank_col#21]
+Arguments: [rank_col#21 DESC NULLS LAST], false, 0
 
-(16) Window
-Input [2]: [item_sk#15, rank_col#16]
-Arguments: [rank(rank_col#16) windowspecdefinition(rank_col#16 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#17], [rank_col#16 DESC NULLS LAST]
+(24) WindowGroupLimit
+Input [2]: [item_sk#20, rank_col#21]
+Arguments: [rank_col#21 DESC NULLS LAST], rank(rank_col#21), 10, Final
 
-(17) Filter [codegen id : 7]
-Input [3]: [item_sk#15, rank_col#16, rnk#17]
-Condition : ((rnk#17 < 11) AND isnotnull(item_sk#15))
+(25) Window
+Input [2]: [item_sk#20, rank_col#21]
+Arguments: [rank(rank_col#21) windowspecdefinition(rank_col#21 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#22], [rank_col#21 DESC NULLS LAST]
 
-(18) Project [codegen id : 7]
-Output [2]: [item_sk#15, rnk#17]
-Input [3]: [item_sk#15, rank_col#16, rnk#17]
+(26) Filter [codegen id : 7]
+Input [3]: [item_sk#20, rank_col#21, rnk#22]
+Condition : ((rnk#22 < 11) AND isnotnull(item_sk#20))
 
-(19) BroadcastExchange
-Input [2]: [item_sk#15, rnk#17]
-Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [plan_id=3]
+(27) Project [codegen id : 7]
+Output [2]: [item_sk#20, rnk#22]
+Input [3]: [item_sk#20, rank_col#21, rnk#22]
 
-(20) BroadcastHashJoin [codegen id : 10]
+(28) BroadcastExchange
+Input [2]: [item_sk#20, rnk#22]
+Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [plan_id=4]
+
+(29) BroadcastHashJoin [codegen id : 10]
 Left keys [1]: [rnk#14]
-Right keys [1]: [rnk#17]
+Right keys [1]: [rnk#22]
 Join type: Inner
 Join condition: None
 
-(21) Project [codegen id : 10]
-Output [3]: [item_sk#10, rnk#14, item_sk#15]
-Input [4]: [item_sk#10, rnk#14, item_sk#15, rnk#17]
+(30) Project [codegen id : 10]
+Output [3]: [item_sk#10, rnk#14, item_sk#20]
+Input [4]: [item_sk#10, rnk#14, item_sk#20, rnk#22]
 
-(22) Scan parquet spark_catalog.default.item
-Output [2]: [i_item_sk#18, i_product_name#19]
+(31) Scan parquet spark_catalog.default.item
+Output [2]: [i_item_sk#23, i_product_name#24]
 Batched: true
 Location [not included in comparison]/{warehouse_dir}/item]
 PushedFilters: [IsNotNull(i_item_sk)]
 ReadSchema: struct<i_item_sk:int,i_product_name:string>
 
-(23) ColumnarToRow [codegen id : 8]
-Input [2]: [i_item_sk#18, i_product_name#19]
+(32) ColumnarToRow [codegen id : 8]
+Input [2]: [i_item_sk#23, i_product_name#24]
 
-(24) Filter [codegen id : 8]
-Input [2]: [i_item_sk#18, i_product_name#19]
-Condition : isnotnull(i_item_sk#18)
+(33) Filter [codegen id : 8]
+Input [2]: [i_item_sk#23, i_product_name#24]
+Condition : isnotnull(i_item_sk#23)
 
-(25) BroadcastExchange
-Input [2]: [i_item_sk#18, i_product_name#19]
-Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4]
+(34) BroadcastExchange
+Input [2]: [i_item_sk#23, i_product_name#24]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5]
 
-(26) BroadcastHashJoin [codegen id : 10]
+(35) BroadcastHashJoin [codegen id : 10]
 Left keys [1]: [item_sk#10]
-Right keys [1]: [i_item_sk#18]
+Right keys [1]: [i_item_sk#23]
 Join type: Inner
 Join condition: None
 
-(27) Project [codegen id : 10]
-Output [3]: [rnk#14, item_sk#15, i_product_name#19]
-Input [5]: [item_sk#10, rnk#14, item_sk#15, i_item_sk#18, i_product_name#19]
+(36) Project [codegen id : 10]
+Output [3]: [rnk#14, item_sk#20, i_product_name#24]
+Input [5]: [item_sk#10, rnk#14, item_sk#20, i_item_sk#23, i_product_name#24]
 
-(28) ReusedExchange [Reuses operator id: 25]
-Output [2]: [i_item_sk#20, i_product_name#21]
+(37) ReusedExchange [Reuses operator id: 34]
+Output [2]: [i_item_sk#25, i_product_name#26]
 
-(29) BroadcastHashJoin [codegen id : 10]
-Left keys [1]: [item_sk#15]
-Right keys [1]: [i_item_sk#20]
+(38) BroadcastHashJoin [codegen id : 10]
+Left keys [1]: [item_sk#20]
+Right keys [1]: [i_item_sk#25]
 Join type: Inner
 Join condition: None
 
-(30) Project [codegen id : 10]
-Output [3]: [rnk#14, i_product_name#19 AS best_performing#22, i_product_name#21 AS worst_performing#23]
-Input [5]: [rnk#14, item_sk#15, i_product_name#19, i_item_sk#20, i_product_name#21]
+(39) Project [codegen id : 10]
+Output [3]: [rnk#14, i_product_name#24 AS best_performing#27, i_product_name#26 AS worst_performing#28]
+Input [5]: [rnk#14, item_sk#20, i_product_name#24, i_item_sk#25, i_product_name#26]
 
-(31) TakeOrderedAndProject
-Input [3]: [rnk#14, best_performing#22, worst_performing#23]
-Arguments: 100, [rnk#14 ASC NULLS FIRST], [rnk#14, best_performing#22, worst_performing#23]
+(40) TakeOrderedAndProject
+Input [3]: [rnk#14, best_performing#27, worst_performing#28]
+Arguments: 100, [rnk#14 ASC NULLS FIRST], [rnk#14, best_performing#27, worst_performing#28]
 
 ===== Subqueries =====
 
 Subquery:1 Hosting operator id = 8 Hosting Expression = Subquery scalar-subquery#12, [id=#13]
-* HashAggregate (38)
-+- Exchange (37)
-   +- * HashAggregate (36)
-      +- * Project (35)
-         +- * Filter (34)
-            +- * ColumnarToRow (33)
-               +- Scan parquet spark_catalog.default.store_sales (32)
+* HashAggregate (47)
++- Exchange (46)
+   +- * HashAggregate (45)
+      +- * Project (44)
+         +- * Filter (43)
+            +- * ColumnarToRow (42)
+               +- Scan parquet spark_catalog.default.store_sales (41)
 
 
-(32) Scan parquet spark_catalog.default.store_sales
-Output [4]: [ss_addr_sk#24, ss_store_sk#25, ss_net_profit#26, ss_sold_date_sk#27]
+(41) Scan parquet spark_catalog.default.store_sales
+Output [4]: [ss_addr_sk#29, ss_store_sk#30, ss_net_profit#31, ss_sold_date_sk#32]
 Batched: true
 Location [not included in comparison]/{warehouse_dir}/store_sales]
 PushedFilters: [IsNotNull(ss_store_sk), EqualTo(ss_store_sk,4), IsNull(ss_addr_sk)]
 ReadSchema: struct<ss_addr_sk:int,ss_store_sk:int,ss_net_profit:decimal(7,2)>
 
-(33) ColumnarToRow [codegen id : 1]
-Input [4]: [ss_addr_sk#24, ss_store_sk#25, ss_net_profit#26, ss_sold_date_sk#27]
-
-(34) Filter [codegen id : 1]
-Input [4]: [ss_addr_sk#24, ss_store_sk#25, ss_net_profit#26, ss_sold_date_sk#27]
-Condition : ((isnotnull(ss_store_sk#25) AND (ss_store_sk#25 = 4)) AND isnull(ss_addr_sk#24))
-
-(35) Project [codegen id : 1]
-Output [2]: [ss_store_sk#25, ss_net_profit#26]
-Input [4]: [ss_addr_sk#24, ss_store_sk#25, ss_net_profit#26, ss_sold_date_sk#27]
-
-(36) HashAggregate [codegen id : 1]
-Input [2]: [ss_store_sk#25, ss_net_profit#26]
-Keys [1]: [ss_store_sk#25]
-Functions [1]: [partial_avg(UnscaledValue(ss_net_profit#26))]
-Aggregate Attributes [2]: [sum#28, count#29]
-Results [3]: [ss_store_sk#25, sum#30, count#31]
-
-(37) Exchange
-Input [3]: [ss_store_sk#25, sum#30, count#31]
-Arguments: hashpartitioning(ss_store_sk#25, 5), ENSURE_REQUIREMENTS, [plan_id=5]
-
-(38) HashAggregate [codegen id : 2]
-Input [3]: [ss_store_sk#25, sum#30, count#31]
-Keys [1]: [ss_store_sk#25]
-Functions [1]: [avg(UnscaledValue(ss_net_profit#26))]
-Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_profit#26))#32]
-Results [1]: [cast((avg(UnscaledValue(ss_net_profit#26))#32 / 100.0) as decimal(11,6)) AS rank_col#33]
+(42) ColumnarToRow [codegen id : 1]
+Input [4]: [ss_addr_sk#29, ss_store_sk#30, ss_net_profit#31, ss_sold_date_sk#32]
+
+(43) Filter [codegen id : 1]
+Input [4]: [ss_addr_sk#29, ss_store_sk#30, ss_net_profit#31, ss_sold_date_sk#32]
+Condition : ((isnotnull(ss_store_sk#30) AND (ss_store_sk#30 = 4)) AND isnull(ss_addr_sk#29))
+
+(44) Project [codegen id : 1]
+Output [2]: [ss_store_sk#30, ss_net_profit#31]
+Input [4]: [ss_addr_sk#29, ss_store_sk#30, ss_net_profit#31, ss_sold_date_sk#32]
+
+(45) HashAggregate [codegen id : 1]
+Input [2]: [ss_store_sk#30, ss_net_profit#31]
+Keys [1]: [ss_store_sk#30]
+Functions [1]: [partial_avg(UnscaledValue(ss_net_profit#31))]
+Aggregate Attributes [2]: [sum#33, count#34]
+Results [3]: [ss_store_sk#30, sum#35, count#36]
+
+(46) Exchange
+Input [3]: [ss_store_sk#30, sum#35, count#36]
+Arguments: hashpartitioning(ss_store_sk#30, 5), ENSURE_REQUIREMENTS, [plan_id=6]
+
+(47) HashAggregate [codegen id : 2]
+Input [3]: [ss_store_sk#30, sum#35, count#36]
+Keys [1]: [ss_store_sk#30]
+Functions [1]: [avg(UnscaledValue(ss_net_profit#31))]
+Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_profit#31))#37]
+Results [1]: [cast((avg(UnscaledValue(ss_net_profit#31))#37 / 100.0) as decimal(11,6)) AS rank_col#38]
+
+Subquery:2 Hosting operator id = 19 Hosting Expression = ReusedSubquery Subquery scalar-subquery#12, [id=#13]
 
 
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.sf100/simplified.txt
index 8d23219df4b..7bdf28429ea 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.sf100/simplified.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.sf100/simplified.txt
@@ -10,34 +10,37 @@ TakeOrderedAndProject [rnk,best_performing,worst_performing]
                   Filter [rnk,item_sk]
                     InputAdapter
                       Window [rank_col]
-                        WholeStageCodegen (3)
-                          Sort [rank_col]
-                            InputAdapter
-                              Exchange #1
-                                WholeStageCodegen (2)
-                                  Filter [rank_col]
-                                    Subquery #1
-                                      WholeStageCodegen (2)
-                                        HashAggregate [ss_store_sk,sum,count] [avg(UnscaledValue(ss_net_profit)),rank_col,sum,count]
-                                          InputAdapter
-                                            Exchange [ss_store_sk] #3
-                                              WholeStageCodegen (1)
-                                                HashAggregate [ss_store_sk,ss_net_profit] [sum,count,sum,count]
-                                                  Project [ss_store_sk,ss_net_profit]
-                                                    Filter [ss_store_sk,ss_addr_sk]
-                                                      ColumnarToRow
-                                                        InputAdapter
-                                                          Scan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk]
-                                    HashAggregate [ss_item_sk,sum,count] [avg(UnscaledValue(ss_net_profit)),item_sk,rank_col,sum,count]
-                                      InputAdapter
-                                        Exchange [ss_item_sk] #2
-                                          WholeStageCodegen (1)
-                                            HashAggregate [ss_item_sk,ss_net_profit] [sum,count,sum,count]
-                                              Project [ss_item_sk,ss_net_profit]
-                                                Filter [ss_store_sk]
-                                                  ColumnarToRow
-                                                    InputAdapter
-                                                      Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk]
+                        WindowGroupLimit [rank_col]
+                          WholeStageCodegen (3)
+                            Sort [rank_col]
+                              InputAdapter
+                                Exchange #1
+                                  WindowGroupLimit [rank_col]
+                                    WholeStageCodegen (2)
+                                      Sort [rank_col]
+                                        Filter [rank_col]
+                                          Subquery #1
+                                            WholeStageCodegen (2)
+                                              HashAggregate [ss_store_sk,sum,count] [avg(UnscaledValue(ss_net_profit)),rank_col,sum,count]
+                                                InputAdapter
+                                                  Exchange [ss_store_sk] #3
+                                                    WholeStageCodegen (1)
+                                                      HashAggregate [ss_store_sk,ss_net_profit] [sum,count,sum,count]
+                                                        Project [ss_store_sk,ss_net_profit]
+                                                          Filter [ss_store_sk,ss_addr_sk]
+                                                            ColumnarToRow
+                                                              InputAdapter
+                                                                Scan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk]
+                                          HashAggregate [ss_item_sk,sum,count] [avg(UnscaledValue(ss_net_profit)),item_sk,rank_col,sum,count]
+                                            InputAdapter
+                                              Exchange [ss_item_sk] #2
+                                                WholeStageCodegen (1)
+                                                  HashAggregate [ss_item_sk,ss_net_profit] [sum,count,sum,count]
+                                                    Project [ss_item_sk,ss_net_profit]
+                                                      Filter [ss_store_sk]
+                                                        ColumnarToRow
+                                                          InputAdapter
+                                                            Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk]
                 InputAdapter
                   BroadcastExchange #4
                     WholeStageCodegen (7)
@@ -45,16 +48,25 @@ TakeOrderedAndProject [rnk,best_performing,worst_performing]
                         Filter [rnk,item_sk]
                           InputAdapter
                             Window [rank_col]
-                              WholeStageCodegen (6)
-                                Sort [rank_col]
-                                  InputAdapter
-                                    ReusedExchange [item_sk,rank_col] #1
+                              WindowGroupLimit [rank_col]
+                                WholeStageCodegen (6)
+                                  Sort [rank_col]
+                                    InputAdapter
+                                      Exchange #5
+                                        WindowGroupLimit [rank_col]
+                                          WholeStageCodegen (5)
+                                            Sort [rank_col]
+                                              Filter [rank_col]
+                                                ReusedSubquery [rank_col] #1
+                                                HashAggregate [ss_item_sk,sum,count] [avg(UnscaledValue(ss_net_profit)),item_sk,rank_col,sum,count]
+                                                  InputAdapter
+                                                    ReusedExchange [ss_item_sk,sum,count] #2
             InputAdapter
-              BroadcastExchange #5
+              BroadcastExchange #6
                 WholeStageCodegen (8)
                   Filter [i_item_sk]
                     ColumnarToRow
                       InputAdapter
                         Scan parquet spark_catalog.default.item [i_item_sk,i_product_name]
         InputAdapter
-          ReusedExchange [i_item_sk,i_product_name] #5
+          ReusedExchange [i_item_sk,i_product_name] #6
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/explain.txt
index 4eda34a7e58..a0b3f36c6cb 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/explain.txt
@@ -1,36 +1,45 @@
 == Physical Plan ==
-TakeOrderedAndProject (32)
-+- * Project (31)
-   +- * BroadcastHashJoin Inner BuildRight (30)
-      :- * Project (28)
-      :  +- * BroadcastHashJoin Inner BuildRight (27)
-      :     :- * Project (22)
-      :     :  +- * SortMergeJoin Inner (21)
-      :     :     :- * Sort (14)
-      :     :     :  +- * Project (13)
-      :     :     :     +- * Filter (12)
-      :     :     :        +- Window (11)
-      :     :     :           +- * Sort (10)
-      :     :     :              +- Exchange (9)
-      :     :     :                 +- * Filter (8)
-      :     :     :                    +- * HashAggregate (7)
-      :     :     :                       +- Exchange (6)
-      :     :     :                          +- * HashAggregate (5)
-      :     :     :                             +- * Project (4)
-      :     :     :                                +- * Filter (3)
-      :     :     :                                   +- * ColumnarToRow (2)
-      :     :     :                                      +- Scan parquet spark_catalog.default.store_sales (1)
-      :     :     +- * Sort (20)
-      :     :        +- * Project (19)
-      :     :           +- * Filter (18)
-      :     :              +- Window (17)
-      :     :                 +- * Sort (16)
-      :     :                    +- ReusedExchange (15)
-      :     +- BroadcastExchange (26)
-      :        +- * Filter (25)
-      :           +- * ColumnarToRow (24)
-      :              +- Scan parquet spark_catalog.default.item (23)
-      +- ReusedExchange (29)
+TakeOrderedAndProject (41)
++- * Project (40)
+   +- * BroadcastHashJoin Inner BuildRight (39)
+      :- * Project (37)
+      :  +- * BroadcastHashJoin Inner BuildRight (36)
+      :     :- * Project (31)
+      :     :  +- * SortMergeJoin Inner (30)
+      :     :     :- * Sort (17)
+      :     :     :  +- * Project (16)
+      :     :     :     +- * Filter (15)
+      :     :     :        +- Window (14)
+      :     :     :           +- WindowGroupLimit (13)
+      :     :     :              +- * Sort (12)
+      :     :     :                 +- Exchange (11)
+      :     :     :                    +- WindowGroupLimit (10)
+      :     :     :                       +- * Sort (9)
+      :     :     :                          +- * Filter (8)
+      :     :     :                             +- * HashAggregate (7)
+      :     :     :                                +- Exchange (6)
+      :     :     :                                   +- * HashAggregate (5)
+      :     :     :                                      +- * Project (4)
+      :     :     :                                         +- * Filter (3)
+      :     :     :                                            +- * ColumnarToRow (2)
+      :     :     :                                               +- Scan parquet spark_catalog.default.store_sales (1)
+      :     :     +- * Sort (29)
+      :     :        +- * Project (28)
+      :     :           +- * Filter (27)
+      :     :              +- Window (26)
+      :     :                 +- WindowGroupLimit (25)
+      :     :                    +- * Sort (24)
+      :     :                       +- Exchange (23)
+      :     :                          +- WindowGroupLimit (22)
+      :     :                             +- * Sort (21)
+      :     :                                +- * Filter (20)
+      :     :                                   +- * HashAggregate (19)
+      :     :                                      +- ReusedExchange (18)
+      :     +- BroadcastExchange (35)
+      :        +- * Filter (34)
+      :           +- * ColumnarToRow (33)
+      :              +- Scan parquet spark_catalog.default.item (32)
+      +- ReusedExchange (38)
 
 
 (1) Scan parquet spark_catalog.default.store_sales
@@ -73,154 +82,195 @@ Results [2]: [ss_item_sk#1 AS item_sk#10, cast((avg(UnscaledValue(ss_net_profit#
 Input [2]: [item_sk#10, rank_col#11]
 Condition : (isnotnull(rank_col#11) AND (cast(rank_col#11 as decimal(13,7)) > (0.9 * Subquery scalar-subquery#12, [id=#13])))
 
-(9) Exchange
+(9) Sort [codegen id : 2]
+Input [2]: [item_sk#10, rank_col#11]
+Arguments: [rank_col#11 ASC NULLS FIRST], false, 0
+
+(10) WindowGroupLimit
+Input [2]: [item_sk#10, rank_col#11]
+Arguments: [rank_col#11 ASC NULLS FIRST], rank(rank_col#11), 10, Partial
+
+(11) Exchange
 Input [2]: [item_sk#10, rank_col#11]
 Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=2]
 
-(10) Sort [codegen id : 3]
+(12) Sort [codegen id : 3]
 Input [2]: [item_sk#10, rank_col#11]
 Arguments: [rank_col#11 ASC NULLS FIRST], false, 0
 
-(11) Window
+(13) WindowGroupLimit
+Input [2]: [item_sk#10, rank_col#11]
+Arguments: [rank_col#11 ASC NULLS FIRST], rank(rank_col#11), 10, Final
+
+(14) Window
 Input [2]: [item_sk#10, rank_col#11]
 Arguments: [rank(rank_col#11) windowspecdefinition(rank_col#11 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#14], [rank_col#11 ASC NULLS FIRST]
 
-(12) Filter [codegen id : 4]
+(15) Filter [codegen id : 4]
 Input [3]: [item_sk#10, rank_col#11, rnk#14]
 Condition : ((rnk#14 < 11) AND isnotnull(item_sk#10))
 
-(13) Project [codegen id : 4]
+(16) Project [codegen id : 4]
 Output [2]: [item_sk#10, rnk#14]
 Input [3]: [item_sk#10, rank_col#11, rnk#14]
 
-(14) Sort [codegen id : 4]
+(17) Sort [codegen id : 4]
 Input [2]: [item_sk#10, rnk#14]
 Arguments: [rnk#14 ASC NULLS FIRST], false, 0
 
-(15) ReusedExchange [Reuses operator id: 9]
-Output [2]: [item_sk#15, rank_col#16]
+(18) ReusedExchange [Reuses operator id: 6]
+Output [3]: [ss_item_sk#15, sum#16, count#17]
+
+(19) HashAggregate [codegen id : 6]
+Input [3]: [ss_item_sk#15, sum#16, count#17]
+Keys [1]: [ss_item_sk#15]
+Functions [1]: [avg(UnscaledValue(ss_net_profit#18))]
+Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_profit#18))#19]
+Results [2]: [ss_item_sk#15 AS item_sk#20, cast((avg(UnscaledValue(ss_net_profit#18))#19 / 100.0) as decimal(11,6)) AS rank_col#21]
+
+(20) Filter [codegen id : 6]
+Input [2]: [item_sk#20, rank_col#21]
+Condition : (isnotnull(rank_col#21) AND (cast(rank_col#21 as decimal(13,7)) > (0.9 * ReusedSubquery Subquery scalar-subquery#12, [id=#13])))
+
+(21) Sort [codegen id : 6]
+Input [2]: [item_sk#20, rank_col#21]
+Arguments: [rank_col#21 DESC NULLS LAST], false, 0
+
+(22) WindowGroupLimit
+Input [2]: [item_sk#20, rank_col#21]
+Arguments: [rank_col#21 DESC NULLS LAST], rank(rank_col#21), 10, Partial
 
-(16) Sort [codegen id : 7]
-Input [2]: [item_sk#15, rank_col#16]
-Arguments: [rank_col#16 DESC NULLS LAST], false, 0
+(23) Exchange
+Input [2]: [item_sk#20, rank_col#21]
+Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=3]
 
-(17) Window
-Input [2]: [item_sk#15, rank_col#16]
-Arguments: [rank(rank_col#16) windowspecdefinition(rank_col#16 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#17], [rank_col#16 DESC NULLS LAST]
+(24) Sort [codegen id : 7]
+Input [2]: [item_sk#20, rank_col#21]
+Arguments: [rank_col#21 DESC NULLS LAST], false, 0
 
-(18) Filter [codegen id : 8]
-Input [3]: [item_sk#15, rank_col#16, rnk#17]
-Condition : ((rnk#17 < 11) AND isnotnull(item_sk#15))
+(25) WindowGroupLimit
+Input [2]: [item_sk#20, rank_col#21]
+Arguments: [rank_col#21 DESC NULLS LAST], rank(rank_col#21), 10, Final
 
-(19) Project [codegen id : 8]
-Output [2]: [item_sk#15, rnk#17]
-Input [3]: [item_sk#15, rank_col#16, rnk#17]
+(26) Window
+Input [2]: [item_sk#20, rank_col#21]
+Arguments: [rank(rank_col#21) windowspecdefinition(rank_col#21 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#22], [rank_col#21 DESC NULLS LAST]
 
-(20) Sort [codegen id : 8]
-Input [2]: [item_sk#15, rnk#17]
-Arguments: [rnk#17 ASC NULLS FIRST], false, 0
+(27) Filter [codegen id : 8]
+Input [3]: [item_sk#20, rank_col#21, rnk#22]
+Condition : ((rnk#22 < 11) AND isnotnull(item_sk#20))
 
-(21) SortMergeJoin [codegen id : 11]
+(28) Project [codegen id : 8]
+Output [2]: [item_sk#20, rnk#22]
+Input [3]: [item_sk#20, rank_col#21, rnk#22]
+
+(29) Sort [codegen id : 8]
+Input [2]: [item_sk#20, rnk#22]
+Arguments: [rnk#22 ASC NULLS FIRST], false, 0
+
+(30) SortMergeJoin [codegen id : 11]
 Left keys [1]: [rnk#14]
-Right keys [1]: [rnk#17]
+Right keys [1]: [rnk#22]
 Join type: Inner
 Join condition: None
 
-(22) Project [codegen id : 11]
-Output [3]: [item_sk#10, rnk#14, item_sk#15]
-Input [4]: [item_sk#10, rnk#14, item_sk#15, rnk#17]
+(31) Project [codegen id : 11]
+Output [3]: [item_sk#10, rnk#14, item_sk#20]
+Input [4]: [item_sk#10, rnk#14, item_sk#20, rnk#22]
 
-(23) Scan parquet spark_catalog.default.item
-Output [2]: [i_item_sk#18, i_product_name#19]
+(32) Scan parquet spark_catalog.default.item
+Output [2]: [i_item_sk#23, i_product_name#24]
 Batched: true
 Location [not included in comparison]/{warehouse_dir}/item]
 PushedFilters: [IsNotNull(i_item_sk)]
 ReadSchema: struct<i_item_sk:int,i_product_name:string>
 
-(24) ColumnarToRow [codegen id : 9]
-Input [2]: [i_item_sk#18, i_product_name#19]
+(33) ColumnarToRow [codegen id : 9]
+Input [2]: [i_item_sk#23, i_product_name#24]
 
-(25) Filter [codegen id : 9]
-Input [2]: [i_item_sk#18, i_product_name#19]
-Condition : isnotnull(i_item_sk#18)
+(34) Filter [codegen id : 9]
+Input [2]: [i_item_sk#23, i_product_name#24]
+Condition : isnotnull(i_item_sk#23)
 
-(26) BroadcastExchange
-Input [2]: [i_item_sk#18, i_product_name#19]
-Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3]
+(35) BroadcastExchange
+Input [2]: [i_item_sk#23, i_product_name#24]
+Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4]
 
-(27) BroadcastHashJoin [codegen id : 11]
+(36) BroadcastHashJoin [codegen id : 11]
 Left keys [1]: [item_sk#10]
-Right keys [1]: [i_item_sk#18]
+Right keys [1]: [i_item_sk#23]
 Join type: Inner
 Join condition: None
 
-(28) Project [codegen id : 11]
-Output [3]: [rnk#14, item_sk#15, i_product_name#19]
-Input [5]: [item_sk#10, rnk#14, item_sk#15, i_item_sk#18, i_product_name#19]
+(37) Project [codegen id : 11]
+Output [3]: [rnk#14, item_sk#20, i_product_name#24]
+Input [5]: [item_sk#10, rnk#14, item_sk#20, i_item_sk#23, i_product_name#24]
 
-(29) ReusedExchange [Reuses operator id: 26]
-Output [2]: [i_item_sk#20, i_product_name#21]
+(38) ReusedExchange [Reuses operator id: 35]
+Output [2]: [i_item_sk#25, i_product_name#26]
 
-(30) BroadcastHashJoin [codegen id : 11]
-Left keys [1]: [item_sk#15]
-Right keys [1]: [i_item_sk#20]
+(39) BroadcastHashJoin [codegen id : 11]
+Left keys [1]: [item_sk#20]
+Right keys [1]: [i_item_sk#25]
 Join type: Inner
 Join condition: None
 
-(31) Project [codegen id : 11]
-Output [3]: [rnk#14, i_product_name#19 AS best_performing#22, i_product_name#21 AS worst_performing#23]
-Input [5]: [rnk#14, item_sk#15, i_product_name#19, i_item_sk#20, i_product_name#21]
+(40) Project [codegen id : 11]
+Output [3]: [rnk#14, i_product_name#24 AS best_performing#27, i_product_name#26 AS worst_performing#28]
+Input [5]: [rnk#14, item_sk#20, i_product_name#24, i_item_sk#25, i_product_name#26]
 
-(32) TakeOrderedAndProject
-Input [3]: [rnk#14, best_performing#22, worst_performing#23]
-Arguments: 100, [rnk#14 ASC NULLS FIRST], [rnk#14, best_performing#22, worst_performing#23]
+(41) TakeOrderedAndProject
+Input [3]: [rnk#14, best_performing#27, worst_performing#28]
+Arguments: 100, [rnk#14 ASC NULLS FIRST], [rnk#14, best_performing#27, worst_performing#28]
 
 ===== Subqueries =====
 
 Subquery:1 Hosting operator id = 8 Hosting Expression = Subquery scalar-subquery#12, [id=#13]
-* HashAggregate (39)
-+- Exchange (38)
-   +- * HashAggregate (37)
-      +- * Project (36)
-         +- * Filter (35)
-            +- * ColumnarToRow (34)
-               +- Scan parquet spark_catalog.default.store_sales (33)
+* HashAggregate (48)
++- Exchange (47)
+   +- * HashAggregate (46)
+      +- * Project (45)
+         +- * Filter (44)
+            +- * ColumnarToRow (43)
+               +- Scan parquet spark_catalog.default.store_sales (42)
 
 
-(33) Scan parquet spark_catalog.default.store_sales
-Output [4]: [ss_addr_sk#24, ss_store_sk#25, ss_net_profit#26, ss_sold_date_sk#27]
+(42) Scan parquet spark_catalog.default.store_sales
+Output [4]: [ss_addr_sk#29, ss_store_sk#30, ss_net_profit#31, ss_sold_date_sk#32]
 Batched: true
 Location [not included in comparison]/{warehouse_dir}/store_sales]
 PushedFilters: [IsNotNull(ss_store_sk), EqualTo(ss_store_sk,4), IsNull(ss_addr_sk)]
 ReadSchema: struct<ss_addr_sk:int,ss_store_sk:int,ss_net_profit:decimal(7,2)>
 
-(34) ColumnarToRow [codegen id : 1]
-Input [4]: [ss_addr_sk#24, ss_store_sk#25, ss_net_profit#26, ss_sold_date_sk#27]
-
-(35) Filter [codegen id : 1]
-Input [4]: [ss_addr_sk#24, ss_store_sk#25, ss_net_profit#26, ss_sold_date_sk#27]
-Condition : ((isnotnull(ss_store_sk#25) AND (ss_store_sk#25 = 4)) AND isnull(ss_addr_sk#24))
-
-(36) Project [codegen id : 1]
-Output [2]: [ss_store_sk#25, ss_net_profit#26]
-Input [4]: [ss_addr_sk#24, ss_store_sk#25, ss_net_profit#26, ss_sold_date_sk#27]
-
-(37) HashAggregate [codegen id : 1]
-Input [2]: [ss_store_sk#25, ss_net_profit#26]
-Keys [1]: [ss_store_sk#25]
-Functions [1]: [partial_avg(UnscaledValue(ss_net_profit#26))]
-Aggregate Attributes [2]: [sum#28, count#29]
-Results [3]: [ss_store_sk#25, sum#30, count#31]
-
-(38) Exchange
-Input [3]: [ss_store_sk#25, sum#30, count#31]
-Arguments: hashpartitioning(ss_store_sk#25, 5), ENSURE_REQUIREMENTS, [plan_id=4]
-
-(39) HashAggregate [codegen id : 2]
-Input [3]: [ss_store_sk#25, sum#30, count#31]
-Keys [1]: [ss_store_sk#25]
-Functions [1]: [avg(UnscaledValue(ss_net_profit#26))]
-Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_profit#26))#32]
-Results [1]: [cast((avg(UnscaledValue(ss_net_profit#26))#32 / 100.0) as decimal(11,6)) AS rank_col#33]
+(43) ColumnarToRow [codegen id : 1]
+Input [4]: [ss_addr_sk#29, ss_store_sk#30, ss_net_profit#31, ss_sold_date_sk#32]
+
+(44) Filter [codegen id : 1]
+Input [4]: [ss_addr_sk#29, ss_store_sk#30, ss_net_profit#31, ss_sold_date_sk#32]
+Condition : ((isnotnull(ss_store_sk#30) AND (ss_store_sk#30 = 4)) AND isnull(ss_addr_sk#29))
+
+(45) Project [codegen id : 1]
+Output [2]: [ss_store_sk#30, ss_net_profit#31]
+Input [4]: [ss_addr_sk#29, ss_store_sk#30, ss_net_profit#31, ss_sold_date_sk#32]
+
+(46) HashAggregate [codegen id : 1]
+Input [2]: [ss_store_sk#30, ss_net_profit#31]
+Keys [1]: [ss_store_sk#30]
+Functions [1]: [partial_avg(UnscaledValue(ss_net_profit#31))]
+Aggregate Attributes [2]: [sum#33, count#34]
+Results [3]: [ss_store_sk#30, sum#35, count#36]
+
+(47) Exchange
+Input [3]: [ss_store_sk#30, sum#35, count#36]
+Arguments: hashpartitioning(ss_store_sk#30, 5), ENSURE_REQUIREMENTS, [plan_id=5]
+
+(48) HashAggregate [codegen id : 2]
+Input [3]: [ss_store_sk#30, sum#35, count#36]
+Keys [1]: [ss_store_sk#30]
+Functions [1]: [avg(UnscaledValue(ss_net_profit#31))]
+Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_profit#31))#37]
+Results [1]: [cast((avg(UnscaledValue(ss_net_profit#31))#37 / 100.0) as decimal(11,6)) AS rank_col#38]
+
+Subquery:2 Hosting operator id = 20 Hosting Expression = ReusedSubquery Subquery scalar-subquery#12, [id=#13]
 
 
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/simplified.txt
index 46d14b650bf..64c8c2506ee 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/simplified.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/simplified.txt
@@ -13,34 +13,37 @@ TakeOrderedAndProject [rnk,best_performing,worst_performing]
                         Filter [rnk,item_sk]
                           InputAdapter
                             Window [rank_col]
-                              WholeStageCodegen (3)
-                                Sort [rank_col]
-                                  InputAdapter
-                                    Exchange #1
-                                      WholeStageCodegen (2)
-                                        Filter [rank_col]
-                                          Subquery #1
-                                            WholeStageCodegen (2)
-                                              HashAggregate [ss_store_sk,sum,count] [avg(UnscaledValue(ss_net_profit)),rank_col,sum,count]
-                                                InputAdapter
-                                                  Exchange [ss_store_sk] #3
-                                                    WholeStageCodegen (1)
-                                                      HashAggregate [ss_store_sk,ss_net_profit] [sum,count,sum,count]
-                                                        Project [ss_store_sk,ss_net_profit]
-                                                          Filter [ss_store_sk,ss_addr_sk]
-                                                            ColumnarToRow
-                                                              InputAdapter
-                                                                Scan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk]
-                                          HashAggregate [ss_item_sk,sum,count] [avg(UnscaledValue(ss_net_profit)),item_sk,rank_col,sum,count]
-                                            InputAdapter
-                                              Exchange [ss_item_sk] #2
-                                                WholeStageCodegen (1)
-                                                  HashAggregate [ss_item_sk,ss_net_profit] [sum,count,sum,count]
-                                                    Project [ss_item_sk,ss_net_profit]
-                                                      Filter [ss_store_sk]
-                                                        ColumnarToRow
-                                                          InputAdapter
-                                                            Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk]
+                              WindowGroupLimit [rank_col]
+                                WholeStageCodegen (3)
+                                  Sort [rank_col]
+                                    InputAdapter
+                                      Exchange #1
+                                        WindowGroupLimit [rank_col]
+                                          WholeStageCodegen (2)
+                                            Sort [rank_col]
+                                              Filter [rank_col]
+                                                Subquery #1
+                                                  WholeStageCodegen (2)
+                                                    HashAggregate [ss_store_sk,sum,count] [avg(UnscaledValue(ss_net_profit)),rank_col,sum,count]
+                                                      InputAdapter
+                                                        Exchange [ss_store_sk] #3
+                                                          WholeStageCodegen (1)
+                                                            HashAggregate [ss_store_sk,ss_net_profit] [sum,count,sum,count]
+                                                              Project [ss_store_sk,ss_net_profit]
+                                                                Filter [ss_store_sk,ss_addr_sk]
+                                                                  ColumnarToRow
+                                                                    InputAdapter
+                                                                      Scan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk]
+                                                HashAggregate [ss_item_sk,sum,count] [avg(UnscaledValue(ss_net_profit)),item_sk,rank_col,sum,count]
+                                                  InputAdapter
+                                                    Exchange [ss_item_sk] #2
+                                                      WholeStageCodegen (1)
+                                                        HashAggregate [ss_item_sk,ss_net_profit] [sum,count,sum,count]
+                                                          Project [ss_item_sk,ss_net_profit]
+                                                            Filter [ss_store_sk]
+                                                              ColumnarToRow
+                                                                InputAdapter
+                                                                  Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk]
                 InputAdapter
                   WholeStageCodegen (8)
                     Sort [rnk]
@@ -48,16 +51,25 @@ TakeOrderedAndProject [rnk,best_performing,worst_performing]
                         Filter [rnk,item_sk]
                           InputAdapter
                             Window [rank_col]
-                              WholeStageCodegen (7)
-                                Sort [rank_col]
-                                  InputAdapter
-                                    ReusedExchange [item_sk,rank_col] #1
+                              WindowGroupLimit [rank_col]
+                                WholeStageCodegen (7)
+                                  Sort [rank_col]
+                                    InputAdapter
+                                      Exchange #4
+                                        WindowGroupLimit [rank_col]
+                                          WholeStageCodegen (6)
+                                            Sort [rank_col]
+                                              Filter [rank_col]
+                                                ReusedSubquery [rank_col] #1
+                                                HashAggregate [ss_item_sk,sum,count] [avg(UnscaledValue(ss_net_profit)),item_sk,rank_col,sum,count]
+                                                  InputAdapter
+                                                    ReusedExchange [ss_item_sk,sum,count] #2
             InputAdapter
-              BroadcastExchange #4
+              BroadcastExchange #5
                 WholeStageCodegen (9)
                   Filter [i_item_sk]
                     ColumnarToRow
                       InputAdapter
                         Scan parquet spark_catalog.default.item [i_item_sk,i_product_name]
         InputAdapter
-          ReusedExchange [i_item_sk,i_product_name] #4
+          ReusedExchange [i_item_sk,i_product_name] #5
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.sf100/explain.txt
index 3f5f8d535d5..aa68bf9ad20 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.sf100/explain.txt
@@ -1,34 +1,37 @@
 == Physical Plan ==
-TakeOrderedAndProject (30)
-+- * Filter (29)
-   +- Window (28)
-      +- * Sort (27)
-         +- Exchange (26)
-            +- * HashAggregate (25)
-               +- Exchange (24)
-                  +- * HashAggregate (23)
-                     +- * Expand (22)
-                        +- * Project (21)
-                           +- * SortMergeJoin Inner (20)
-                              :- * Sort (14)
-                              :  +- Exchange (13)
-                              :     +- * Project (12)
-                              :        +- * BroadcastHashJoin Inner BuildRight (11)
-                              :           :- * Project (6)
-                              :           :  +- * BroadcastHashJoin Inner BuildRight (5)
-                              :           :     :- * Filter (3)
-                              :           :     :  +- * ColumnarToRow (2)
-                              :           :     :     +- Scan parquet spark_catalog.default.store_sales (1)
-                              :           :     +- ReusedExchange (4)
-                              :           +- BroadcastExchange (10)
-                              :              +- * Filter (9)
-                              :                 +- * ColumnarToRow (8)
-                              :                    +- Scan parquet spark_catalog.default.store (7)
-                              +- * Sort (19)
-                                 +- Exchange (18)
-                                    +- * Filter (17)
-                                       +- * ColumnarToRow (16)
-                                          +- Scan parquet spark_catalog.default.item (15)
+TakeOrderedAndProject (33)
++- * Filter (32)
+   +- Window (31)
+      +- WindowGroupLimit (30)
+         +- * Sort (29)
+            +- Exchange (28)
+               +- WindowGroupLimit (27)
+                  +- * Sort (26)
+                     +- * HashAggregate (25)
+                        +- Exchange (24)
+                           +- * HashAggregate (23)
+                              +- * Expand (22)
+                                 +- * Project (21)
+                                    +- * SortMergeJoin Inner (20)
+                                       :- * Sort (14)
+                                       :  +- Exchange (13)
+                                       :     +- * Project (12)
+                                       :        +- * BroadcastHashJoin Inner BuildRight (11)
+                                       :           :- * Project (6)
+                                       :           :  +- * BroadcastHashJoin Inner BuildRight (5)
+                                       :           :     :- * Filter (3)
+                                       :           :     :  +- * ColumnarToRow (2)
+                                       :           :     :     +- Scan parquet spark_catalog.default.store_sales (1)
+                                       :           :     +- ReusedExchange (4)
+                                       :           +- BroadcastExchange (10)
+                                       :              +- * Filter (9)
+                                       :                 +- * ColumnarToRow (8)
+                                       :                    +- Scan parquet spark_catalog.default.store (7)
+                                       +- * Sort (19)
+                                          +- Exchange (18)
+                                             +- * Filter (17)
+                                                +- * ColumnarToRow (16)
+                                                   +- Scan parquet spark_catalog.default.item (15)
 
 
 (1) Scan parquet spark_catalog.default.store_sales
@@ -46,7 +49,7 @@ Input [5]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sol
 Input [5]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5]
 Condition : (isnotnull(ss_store_sk#2) AND isnotnull(ss_item_sk#1))
 
-(4) ReusedExchange [Reuses operator id: 35]
+(4) ReusedExchange [Reuses operator id: 38]
 Output [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10]
 
 (5) BroadcastHashJoin [codegen id : 3]
@@ -149,55 +152,67 @@ Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(1
 Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#31]
 Results [9]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#31 AS sumsales#32]
 
-(26) Exchange
+(26) Sort [codegen id : 8]
+Input [9]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sumsales#32]
+Arguments: [i_category#18 ASC NULLS FIRST, sumsales#32 DESC NULLS LAST], false, 0
+
+(27) WindowGroupLimit
+Input [9]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sumsales#32]
+Arguments: [i_category#18], [sumsales#32 DESC NULLS LAST], rank(sumsales#32), 100, Partial
+
+(28) Exchange
 Input [9]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sumsales#32]
 Arguments: hashpartitioning(i_category#18, 5), ENSURE_REQUIREMENTS, [plan_id=5]
 
-(27) Sort [codegen id : 9]
+(29) Sort [codegen id : 9]
 Input [9]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sumsales#32]
 Arguments: [i_category#18 ASC NULLS FIRST, sumsales#32 DESC NULLS LAST], false, 0
 
-(28) Window
+(30) WindowGroupLimit
+Input [9]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sumsales#32]
+Arguments: [i_category#18], [sumsales#32 DESC NULLS LAST], rank(sumsales#32), 100, Final
+
+(31) Window
 Input [9]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sumsales#32]
 Arguments: [rank(sumsales#32) windowspecdefinition(i_category#18, sumsales#32 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#33], [i_category#18], [sumsales#32 DESC NULLS LAST]
 
-(29) Filter [codegen id : 10]
+(32) Filter [codegen id : 10]
 Input [10]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sumsales#32, rk#33]
 Condition : (rk#33 <= 100)
 
-(30) TakeOrderedAndProject
+(33) TakeOrderedAndProject
 Input [10]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sumsales#32, rk#33]
 Arguments: 100, [i_category#18 ASC NULLS FIRST, i_class#19 ASC NULLS FIRST, i_brand#20 ASC NULLS FIRST, i_product_name#21 ASC NULLS FIRST, d_year#22 ASC NULLS FIRST, d_qoy#23 ASC NULLS FIRST, d_moy#24 ASC NULLS FIRST, s_store_id#25 ASC NULLS FIRST, sumsales#32 ASC NULLS FIRST, rk#33 ASC NULLS FIRST], [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sumsales#32, rk#33]
 
 ===== Subqueries =====
 
 Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6
-BroadcastExchange (35)
-+- * Project (34)
-   +- * Filter (33)
-      +- * ColumnarToRow (32)
-         +- Scan parquet spark_catalog.default.date_dim (31)
+BroadcastExchange (38)
++- * Project (37)
+   +- * Filter (36)
+      +- * ColumnarToRow (35)
+         +- Scan parquet spark_catalog.default.date_dim (34)
 
 
-(31) Scan parquet spark_catalog.default.date_dim
+(34) Scan parquet spark_catalog.default.date_dim
 Output [5]: [d_date_sk#7, d_month_seq#34, d_year#8, d_moy#9, d_qoy#10]
 Batched: true
 Location [not included in comparison]/{warehouse_dir}/date_dim]
 PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)]
 ReadSchema: struct<d_date_sk:int,d_month_seq:int,d_year:int,d_moy:int,d_qoy:int>
 
-(32) ColumnarToRow [codegen id : 1]
+(35) ColumnarToRow [codegen id : 1]
 Input [5]: [d_date_sk#7, d_month_seq#34, d_year#8, d_moy#9, d_qoy#10]
 
-(33) Filter [codegen id : 1]
+(36) Filter [codegen id : 1]
 Input [5]: [d_date_sk#7, d_month_seq#34, d_year#8, d_moy#9, d_qoy#10]
 Condition : (((isnotnull(d_month_seq#34) AND (d_month_seq#34 >= 1200)) AND (d_month_seq#34 <= 1211)) AND isnotnull(d_date_sk#7))
 
-(34) Project [codegen id : 1]
+(37) Project [codegen id : 1]
 Output [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10]
 Input [5]: [d_date_sk#7, d_month_seq#34, d_year#8, d_moy#9, d_qoy#10]
 
-(35) BroadcastExchange
+(38) BroadcastExchange
 Input [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10]
 Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6]
 
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.sf100/simplified.txt
index 7a3d435fc1b..787b556470c 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.sf100/simplified.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.sf100/simplified.txt
@@ -3,57 +3,60 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_
     Filter [rk]
       InputAdapter
         Window [sumsales,i_category]
-          WholeStageCodegen (9)
-            Sort [i_category,sumsales]
-              InputAdapter
-                Exchange [i_category] #1
-                  WholeStageCodegen (8)
-                    HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty]
-                      InputAdapter
-                        Exchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id] #2
-                          WholeStageCodegen (7)
-                            HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id,ss_sales_price,ss_quantity] [sum,isEmpty,sum,isEmpty]
-                              Expand [ss_quantity,ss_sales_price,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id]
-                                Project [ss_quantity,ss_sales_price,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id]
-                                  SortMergeJoin [ss_item_sk,i_item_sk]
-                                    InputAdapter
-                                      WholeStageCodegen (4)
-                                        Sort [ss_item_sk]
+          WindowGroupLimit [i_category,sumsales]
+            WholeStageCodegen (9)
+              Sort [i_category,sumsales]
+                InputAdapter
+                  Exchange [i_category] #1
+                    WindowGroupLimit [i_category,sumsales]
+                      WholeStageCodegen (8)
+                        Sort [i_category,sumsales]
+                          HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty]
+                            InputAdapter
+                              Exchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id] #2
+                                WholeStageCodegen (7)
+                                  HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id,ss_sales_price,ss_quantity] [sum,isEmpty,sum,isEmpty]
+                                    Expand [ss_quantity,ss_sales_price,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id]
+                                      Project [ss_quantity,ss_sales_price,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id]
+                                        SortMergeJoin [ss_item_sk,i_item_sk]
                                           InputAdapter
-                                            Exchange [ss_item_sk] #3
-                                              WholeStageCodegen (3)
-                                                Project [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id]
-                                                  BroadcastHashJoin [ss_store_sk,s_store_sk]
-                                                    Project [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy]
-                                                      BroadcastHashJoin [ss_sold_date_sk,d_date_sk]
-                                                        Filter [ss_store_sk,ss_item_sk]
-                                                          ColumnarToRow
-                                                            InputAdapter
-                                                              Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk]
-                                                                SubqueryBroadcast [d_date_sk] #1
-                                                                  BroadcastExchange #4
-                                                                    WholeStageCodegen (1)
-                                                                      Project [d_date_sk,d_year,d_moy,d_qoy]
-                                                                        Filter [d_month_seq,d_date_sk]
-                                                                          ColumnarToRow
-                                                                            InputAdapter
-                                                                              Scan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy]
-                                                        InputAdapter
-                                                          ReusedExchange [d_date_sk,d_year,d_moy,d_qoy] #4
-                                                    InputAdapter
-                                                      BroadcastExchange #5
-                                                        WholeStageCodegen (2)
-                                                          Filter [s_store_sk]
-                                                            ColumnarToRow
+                                            WholeStageCodegen (4)
+                                              Sort [ss_item_sk]
+                                                InputAdapter
+                                                  Exchange [ss_item_sk] #3
+                                                    WholeStageCodegen (3)
+                                                      Project [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id]
+                                                        BroadcastHashJoin [ss_store_sk,s_store_sk]
+                                                          Project [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy]
+                                                            BroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+                                                              Filter [ss_store_sk,ss_item_sk]
+                                                                ColumnarToRow
+                                                                  InputAdapter
+                                                                    Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk]
+                                                                      SubqueryBroadcast [d_date_sk] #1
+                                                                        BroadcastExchange #4
+                                                                          WholeStageCodegen (1)
+                                                                            Project [d_date_sk,d_year,d_moy,d_qoy]
+                                                                              Filter [d_month_seq,d_date_sk]
+                                                                                ColumnarToRow
+                                                                                  InputAdapter
+                                                                                    Scan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy]
                                                               InputAdapter
-                                                                Scan parquet spark_catalog.default.store [s_store_sk,s_store_id]
-                                    InputAdapter
-                                      WholeStageCodegen (6)
-                                        Sort [i_item_sk]
+                                                                ReusedExchange [d_date_sk,d_year,d_moy,d_qoy] #4
+                                                          InputAdapter
+                                                            BroadcastExchange #5
+                                                              WholeStageCodegen (2)
+                                                                Filter [s_store_sk]
+                                                                  ColumnarToRow
+                                                                    InputAdapter
+                                                                      Scan parquet spark_catalog.default.store [s_store_sk,s_store_id]
                                           InputAdapter
-                                            Exchange [i_item_sk] #6
-                                              WholeStageCodegen (5)
-                                                Filter [i_item_sk]
-                                                  ColumnarToRow
-                                                    InputAdapter
-                                                      Scan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name]
+                                            WholeStageCodegen (6)
+                                              Sort [i_item_sk]
+                                                InputAdapter
+                                                  Exchange [i_item_sk] #6
+                                                    WholeStageCodegen (5)
+                                                      Filter [i_item_sk]
+                                                        ColumnarToRow
+                                                          InputAdapter
+                                                            Scan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name]
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/explain.txt
index 69f35fdca1e..abb794e3e27 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/explain.txt
@@ -1,31 +1,34 @@
 == Physical Plan ==
-TakeOrderedAndProject (27)
-+- * Filter (26)
-   +- Window (25)
-      +- * Sort (24)
-         +- Exchange (23)
-            +- * HashAggregate (22)
-               +- Exchange (21)
-                  +- * HashAggregate (20)
-                     +- * Expand (19)
-                        +- * Project (18)
-                           +- * BroadcastHashJoin Inner BuildRight (17)
-                              :- * Project (12)
-                              :  +- * BroadcastHashJoin Inner BuildRight (11)
-                              :     :- * Project (6)
-                              :     :  +- * BroadcastHashJoin Inner BuildRight (5)
-                              :     :     :- * Filter (3)
-                              :     :     :  +- * ColumnarToRow (2)
-                              :     :     :     +- Scan parquet spark_catalog.default.store_sales (1)
-                              :     :     +- ReusedExchange (4)
-                              :     +- BroadcastExchange (10)
-                              :        +- * Filter (9)
-                              :           +- * ColumnarToRow (8)
-                              :              +- Scan parquet spark_catalog.default.store (7)
-                              +- BroadcastExchange (16)
-                                 +- * Filter (15)
-                                    +- * ColumnarToRow (14)
-                                       +- Scan parquet spark_catalog.default.item (13)
+TakeOrderedAndProject (30)
++- * Filter (29)
+   +- Window (28)
+      +- WindowGroupLimit (27)
+         +- * Sort (26)
+            +- Exchange (25)
+               +- WindowGroupLimit (24)
+                  +- * Sort (23)
+                     +- * HashAggregate (22)
+                        +- Exchange (21)
+                           +- * HashAggregate (20)
+                              +- * Expand (19)
+                                 +- * Project (18)
+                                    +- * BroadcastHashJoin Inner BuildRight (17)
+                                       :- * Project (12)
+                                       :  +- * BroadcastHashJoin Inner BuildRight (11)
+                                       :     :- * Project (6)
+                                       :     :  +- * BroadcastHashJoin Inner BuildRight (5)
+                                       :     :     :- * Filter (3)
+                                       :     :     :  +- * ColumnarToRow (2)
+                                       :     :     :     +- Scan parquet spark_catalog.default.store_sales (1)
+                                       :     :     +- ReusedExchange (4)
+                                       :     +- BroadcastExchange (10)
+                                       :        +- * Filter (9)
+                                       :           +- * ColumnarToRow (8)
+                                       :              +- Scan parquet spark_catalog.default.store (7)
+                                       +- BroadcastExchange (16)
+                                          +- * Filter (15)
+                                             +- * ColumnarToRow (14)
+                                                +- Scan parquet spark_catalog.default.item (13)
 
 
 (1) Scan parquet spark_catalog.default.store_sales
@@ -43,7 +46,7 @@ Input [5]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sol
 Input [5]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5]
 Condition : (isnotnull(ss_store_sk#2) AND isnotnull(ss_item_sk#1))
 
-(4) ReusedExchange [Reuses operator id: 32]
+(4) ReusedExchange [Reuses operator id: 35]
 Output [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10]
 
 (5) BroadcastHashJoin [codegen id : 4]
@@ -134,55 +137,67 @@ Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(1
 Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#31]
 Results [9]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#31 AS sumsales#32]
 
-(23) Exchange
+(23) Sort [codegen id : 5]
+Input [9]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sumsales#32]
+Arguments: [i_category#18 ASC NULLS FIRST, sumsales#32 DESC NULLS LAST], false, 0
+
+(24) WindowGroupLimit
+Input [9]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sumsales#32]
+Arguments: [i_category#18], [sumsales#32 DESC NULLS LAST], rank(sumsales#32), 100, Partial
+
+(25) Exchange
 Input [9]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sumsales#32]
 Arguments: hashpartitioning(i_category#18, 5), ENSURE_REQUIREMENTS, [plan_id=4]
 
-(24) Sort [codegen id : 6]
+(26) Sort [codegen id : 6]
 Input [9]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sumsales#32]
 Arguments: [i_category#18 ASC NULLS FIRST, sumsales#32 DESC NULLS LAST], false, 0
 
-(25) Window
+(27) WindowGroupLimit
+Input [9]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sumsales#32]
+Arguments: [i_category#18], [sumsales#32 DESC NULLS LAST], rank(sumsales#32), 100, Final
+
+(28) Window
 Input [9]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sumsales#32]
 Arguments: [rank(sumsales#32) windowspecdefinition(i_category#18, sumsales#32 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#33], [i_category#18], [sumsales#32 DESC NULLS LAST]
 
-(26) Filter [codegen id : 7]
+(29) Filter [codegen id : 7]
 Input [10]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sumsales#32, rk#33]
 Condition : (rk#33 <= 100)
 
-(27) TakeOrderedAndProject
+(30) TakeOrderedAndProject
 Input [10]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sumsales#32, rk#33]
 Arguments: 100, [i_category#18 ASC NULLS FIRST, i_class#19 ASC NULLS FIRST, i_brand#20 ASC NULLS FIRST, i_product_name#21 ASC NULLS FIRST, d_year#22 ASC NULLS FIRST, d_qoy#23 ASC NULLS FIRST, d_moy#24 ASC NULLS FIRST, s_store_id#25 ASC NULLS FIRST, sumsales#32 ASC NULLS FIRST, rk#33 ASC NULLS FIRST], [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sumsales#32, rk#33]
 
 ===== Subqueries =====
 
 Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6
-BroadcastExchange (32)
-+- * Project (31)
-   +- * Filter (30)
-      +- * ColumnarToRow (29)
-         +- Scan parquet spark_catalog.default.date_dim (28)
+BroadcastExchange (35)
++- * Project (34)
+   +- * Filter (33)
+      +- * ColumnarToRow (32)
+         +- Scan parquet spark_catalog.default.date_dim (31)
 
 
-(28) Scan parquet spark_catalog.default.date_dim
+(31) Scan parquet spark_catalog.default.date_dim
 Output [5]: [d_date_sk#7, d_month_seq#34, d_year#8, d_moy#9, d_qoy#10]
 Batched: true
 Location [not included in comparison]/{warehouse_dir}/date_dim]
 PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)]
 ReadSchema: struct<d_date_sk:int,d_month_seq:int,d_year:int,d_moy:int,d_qoy:int>
 
-(29) ColumnarToRow [codegen id : 1]
+(32) ColumnarToRow [codegen id : 1]
 Input [5]: [d_date_sk#7, d_month_seq#34, d_year#8, d_moy#9, d_qoy#10]
 
-(30) Filter [codegen id : 1]
+(33) Filter [codegen id : 1]
 Input [5]: [d_date_sk#7, d_month_seq#34, d_year#8, d_moy#9, d_qoy#10]
 Condition : (((isnotnull(d_month_seq#34) AND (d_month_seq#34 >= 1200)) AND (d_month_seq#34 <= 1211)) AND isnotnull(d_date_sk#7))
 
-(31) Project [codegen id : 1]
+(34) Project [codegen id : 1]
 Output [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10]
 Input [5]: [d_date_sk#7, d_month_seq#34, d_year#8, d_moy#9, d_qoy#10]
 
-(32) BroadcastExchange
+(35) BroadcastExchange
 Input [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10]
 Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5]
 
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/simplified.txt
index eb56d128fd0..8a942a70586 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/simplified.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/simplified.txt
@@ -3,48 +3,51 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_
     Filter [rk]
       InputAdapter
         Window [sumsales,i_category]
-          WholeStageCodegen (6)
-            Sort [i_category,sumsales]
-              InputAdapter
-                Exchange [i_category] #1
-                  WholeStageCodegen (5)
-                    HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty]
-                      InputAdapter
-                        Exchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id] #2
-                          WholeStageCodegen (4)
-                            HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id,ss_sales_price,ss_quantity] [sum,isEmpty,sum,isEmpty]
-                              Expand [ss_quantity,ss_sales_price,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id]
-                                Project [ss_quantity,ss_sales_price,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id]
-                                  BroadcastHashJoin [ss_item_sk,i_item_sk]
-                                    Project [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id]
-                                      BroadcastHashJoin [ss_store_sk,s_store_sk]
-                                        Project [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy]
-                                          BroadcastHashJoin [ss_sold_date_sk,d_date_sk]
-                                            Filter [ss_store_sk,ss_item_sk]
-                                              ColumnarToRow
-                                                InputAdapter
-                                                  Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk]
-                                                    SubqueryBroadcast [d_date_sk] #1
-                                                      BroadcastExchange #3
-                                                        WholeStageCodegen (1)
-                                                          Project [d_date_sk,d_year,d_moy,d_qoy]
-                                                            Filter [d_month_seq,d_date_sk]
-                                                              ColumnarToRow
-                                                                InputAdapter
-                                                                  Scan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy]
-                                            InputAdapter
-                                              ReusedExchange [d_date_sk,d_year,d_moy,d_qoy] #3
-                                        InputAdapter
-                                          BroadcastExchange #4
-                                            WholeStageCodegen (2)
-                                              Filter [s_store_sk]
-                                                ColumnarToRow
+          WindowGroupLimit [i_category,sumsales]
+            WholeStageCodegen (6)
+              Sort [i_category,sumsales]
+                InputAdapter
+                  Exchange [i_category] #1
+                    WindowGroupLimit [i_category,sumsales]
+                      WholeStageCodegen (5)
+                        Sort [i_category,sumsales]
+                          HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty]
+                            InputAdapter
+                              Exchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id] #2
+                                WholeStageCodegen (4)
+                                  HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id,ss_sales_price,ss_quantity] [sum,isEmpty,sum,isEmpty]
+                                    Expand [ss_quantity,ss_sales_price,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id]
+                                      Project [ss_quantity,ss_sales_price,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id]
+                                        BroadcastHashJoin [ss_item_sk,i_item_sk]
+                                          Project [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id]
+                                            BroadcastHashJoin [ss_store_sk,s_store_sk]
+                                              Project [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy]
+                                                BroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+                                                  Filter [ss_store_sk,ss_item_sk]
+                                                    ColumnarToRow
+                                                      InputAdapter
+                                                        Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk]
+                                                          SubqueryBroadcast [d_date_sk] #1
+                                                            BroadcastExchange #3
+                                                              WholeStageCodegen (1)
+                                                                Project [d_date_sk,d_year,d_moy,d_qoy]
+                                                                  Filter [d_month_seq,d_date_sk]
+                                                                    ColumnarToRow
+                                                                      InputAdapter
+                                                                        Scan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy]
                                                   InputAdapter
-                                                    Scan parquet spark_catalog.default.store [s_store_sk,s_store_id]
-                                    InputAdapter
-                                      BroadcastExchange #5
-                                        WholeStageCodegen (3)
-                                          Filter [i_item_sk]
-                                            ColumnarToRow
+                                                    ReusedExchange [d_date_sk,d_year,d_moy,d_qoy] #3
                                               InputAdapter
-                                                Scan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name]
+                                                BroadcastExchange #4
+                                                  WholeStageCodegen (2)
+                                                    Filter [s_store_sk]
+                                                      ColumnarToRow
+                                                        InputAdapter
+                                                          Scan parquet spark_catalog.default.store [s_store_sk,s_store_id]
+                                          InputAdapter
+                                            BroadcastExchange #5
+                                              WholeStageCodegen (3)
+                                                Filter [i_item_sk]
+                                                  ColumnarToRow
+                                                    InputAdapter
+                                                      Scan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name]
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/explain.txt
index c7ca5a5eb1f..d64f560f144 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/explain.txt
@@ -1,46 +1,47 @@
 == Physical Plan ==
-TakeOrderedAndProject (42)
-+- * Project (41)
-   +- Window (40)
-      +- * Sort (39)
-         +- Exchange (38)
-            +- * HashAggregate (37)
-               +- Exchange (36)
-                  +- * HashAggregate (35)
-                     +- * Expand (34)
-                        +- * Project (33)
-                           +- * BroadcastHashJoin Inner BuildRight (32)
+TakeOrderedAndProject (43)
++- * Project (42)
+   +- Window (41)
+      +- * Sort (40)
+         +- Exchange (39)
+            +- * HashAggregate (38)
+               +- Exchange (37)
+                  +- * HashAggregate (36)
+                     +- * Expand (35)
+                        +- * Project (34)
+                           +- * BroadcastHashJoin Inner BuildRight (33)
                               :- * Project (6)
                               :  +- * BroadcastHashJoin Inner BuildRight (5)
                               :     :- * Filter (3)
                               :     :  +- * ColumnarToRow (2)
                               :     :     +- Scan parquet spark_catalog.default.store_sales (1)
                               :     +- ReusedExchange (4)
-                              +- BroadcastExchange (31)
-                                 +- * BroadcastHashJoin LeftSemi BuildRight (30)
+                              +- BroadcastExchange (32)
+                                 +- * BroadcastHashJoin LeftSemi BuildRight (31)
                                     :- * Filter (9)
                                     :  +- * ColumnarToRow (8)
                                     :     +- Scan parquet spark_catalog.default.store (7)
-                                    +- BroadcastExchange (29)
-                                       +- * Project (28)
-                                          +- * Filter (27)
-                                             +- Window (26)
-                                                +- * Sort (25)
-                                                   +- * HashAggregate (24)
-                                                      +- Exchange (23)
-                                                         +- * HashAggregate (22)
-                                                            +- * Project (21)
-                                                               +- * BroadcastHashJoin Inner BuildRight (20)
-                                                                  :- * Project (15)
-                                                                  :  +- * BroadcastHashJoin Inner BuildRight (14)
-                                                                  :     :- * Filter (12)
-                                                                  :     :  +- * ColumnarToRow (11)
-                                                                  :     :     +- Scan parquet spark_catalog.default.store_sales (10)
-                                                                  :     +- ReusedExchange (13)
-                                                                  +- BroadcastExchange (19)
-                                                                     +- * Filter (18)
-                                                                        +- * ColumnarToRow (17)
-                                                                           +- Scan parquet spark_catalog.default.store (16)
+                                    +- BroadcastExchange (30)
+                                       +- * Project (29)
+                                          +- * Filter (28)
+                                             +- Window (27)
+                                                +- WindowGroupLimit (26)
+                                                   +- * Sort (25)
+                                                      +- * HashAggregate (24)
+                                                         +- Exchange (23)
+                                                            +- * HashAggregate (22)
+                                                               +- * Project (21)
+                                                                  +- * BroadcastHashJoin Inner BuildRight (20)
+                                                                     :- * Project (15)
+                                                                     :  +- * BroadcastHashJoin Inner BuildRight (14)
+                                                                     :     :- * Filter (12)
+                                                                     :     :  +- * ColumnarToRow (11)
+                                                                     :     :     +- Scan parquet spark_catalog.default.store_sales (10)
+                                                                     :     +- ReusedExchange (13)
+                                                                     +- BroadcastExchange (19)
+                                                                        +- * Filter (18)
+                                                                           +- * ColumnarToRow (17)
+                                                                              +- Scan parquet spark_catalog.default.store (16)
 
 
 (1) Scan parquet spark_catalog.default.store_sales
@@ -58,7 +59,7 @@ Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3]
 Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3]
 Condition : isnotnull(ss_store_sk#1)
 
-(4) ReusedExchange [Reuses operator id: 47]
+(4) ReusedExchange [Reuses operator id: 48]
 Output [1]: [d_date_sk#5]
 
 (5) BroadcastHashJoin [codegen id : 8]
@@ -100,7 +101,7 @@ Input [3]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11]
 Input [3]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11]
 Condition : isnotnull(ss_store_sk#9)
 
-(13) ReusedExchange [Reuses operator id: 47]
+(13) ReusedExchange [Reuses operator id: 48]
 Output [1]: [d_date_sk#12]
 
 (14) BroadcastHashJoin [codegen id : 4]
@@ -163,113 +164,117 @@ Results [3]: [s_state#14, MakeDecimal(sum(UnscaledValue(ss_net_profit#10))#17,17
 Input [3]: [s_state#14, _w0#18, s_state#14]
 Arguments: [s_state#14 ASC NULLS FIRST, _w0#18 DESC NULLS LAST], false, 0
 
-(26) Window
+(26) WindowGroupLimit
+Input [3]: [s_state#14, _w0#18, s_state#14]
+Arguments: [s_state#14], [_w0#18 DESC NULLS LAST], rank(_w0#18), 5, Final
+
+(27) Window
 Input [3]: [s_state#14, _w0#18, s_state#14]
 Arguments: [rank(_w0#18) windowspecdefinition(s_state#14, _w0#18 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#19], [s_state#14], [_w0#18 DESC NULLS LAST]
 
-(27) Filter [codegen id : 6]
+(28) Filter [codegen id : 6]
 Input [4]: [s_state#14, _w0#18, s_state#14, ranking#19]
 Condition : (ranking#19 <= 5)
 
-(28) Project [codegen id : 6]
+(29) Project [codegen id : 6]
 Output [1]: [s_state#14]
 Input [4]: [s_state#14, _w0#18, s_state#14, ranking#19]
 
-(29) BroadcastExchange
+(30) BroadcastExchange
 Input [1]: [s_state#14]
 Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=3]
 
-(30) BroadcastHashJoin [codegen id : 7]
+(31) BroadcastHashJoin [codegen id : 7]
 Left keys [1]: [s_state#8]
 Right keys [1]: [s_state#14]
 Join type: LeftSemi
 Join condition: None
 
-(31) BroadcastExchange
+(32) BroadcastExchange
 Input [3]: [s_store_sk#6, s_county#7, s_state#8]
 Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4]
 
-(32) BroadcastHashJoin [codegen id : 8]
+(33) BroadcastHashJoin [codegen id : 8]
 Left keys [1]: [ss_store_sk#1]
 Right keys [1]: [s_store_sk#6]
 Join type: Inner
 Join condition: None
 
-(33) Project [codegen id : 8]
+(34) Project [codegen id : 8]
 Output [3]: [ss_net_profit#2, s_state#8, s_county#7]
 Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#6, s_county#7, s_state#8]
 
-(34) Expand [codegen id : 8]
+(35) Expand [codegen id : 8]
 Input [3]: [ss_net_profit#2, s_state#8, s_county#7]
 Arguments: [[ss_net_profit#2, s_state#8, s_county#7, 0], [ss_net_profit#2, s_state#8, null, 1], [ss_net_profit#2, null, null, 3]], [ss_net_profit#2, s_state#20, s_county#21, spark_grouping_id#22]
 
-(35) HashAggregate [codegen id : 8]
+(36) HashAggregate [codegen id : 8]
 Input [4]: [ss_net_profit#2, s_state#20, s_county#21, spark_grouping_id#22]
 Keys [3]: [s_state#20, s_county#21, spark_grouping_id#22]
 Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))]
 Aggregate Attributes [1]: [sum#23]
 Results [4]: [s_state#20, s_county#21, spark_grouping_id#22, sum#24]
 
-(36) Exchange
+(37) Exchange
 Input [4]: [s_state#20, s_county#21, spark_grouping_id#22, sum#24]
 Arguments: hashpartitioning(s_state#20, s_county#21, spark_grouping_id#22, 5), ENSURE_REQUIREMENTS, [plan_id=5]
 
-(37) HashAggregate [codegen id : 9]
+(38) HashAggregate [codegen id : 9]
 Input [4]: [s_state#20, s_county#21, spark_grouping_id#22, sum#24]
 Keys [3]: [s_state#20, s_county#21, spark_grouping_id#22]
 Functions [1]: [sum(UnscaledValue(ss_net_profit#2))]
 Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#25]
 Results [7]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#25,17,2) AS total_sum#26, s_state#20, s_county#21, (cast((shiftright(spark_grouping_id#22, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#22, 0) & 1) as tinyint)) AS lochierarchy#27, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#25,17,2) AS _w0#28, (cast((shiftright(spark_grouping_id#22, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#22, 0) & 1) as tinyint)) AS _w1#29, CASE WHEN (cast((shiftright(spark_gr [...]
 
-(38) Exchange
+(39) Exchange
 Input [7]: [total_sum#26, s_state#20, s_county#21, lochierarchy#27, _w0#28, _w1#29, _w2#30]
 Arguments: hashpartitioning(_w1#29, _w2#30, 5), ENSURE_REQUIREMENTS, [plan_id=6]
 
-(39) Sort [codegen id : 10]
+(40) Sort [codegen id : 10]
 Input [7]: [total_sum#26, s_state#20, s_county#21, lochierarchy#27, _w0#28, _w1#29, _w2#30]
 Arguments: [_w1#29 ASC NULLS FIRST, _w2#30 ASC NULLS FIRST, _w0#28 DESC NULLS LAST], false, 0
 
-(40) Window
+(41) Window
 Input [7]: [total_sum#26, s_state#20, s_county#21, lochierarchy#27, _w0#28, _w1#29, _w2#30]
 Arguments: [rank(_w0#28) windowspecdefinition(_w1#29, _w2#30, _w0#28 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#31], [_w1#29, _w2#30], [_w0#28 DESC NULLS LAST]
 
-(41) Project [codegen id : 11]
+(42) Project [codegen id : 11]
 Output [5]: [total_sum#26, s_state#20, s_county#21, lochierarchy#27, rank_within_parent#31]
 Input [8]: [total_sum#26, s_state#20, s_county#21, lochierarchy#27, _w0#28, _w1#29, _w2#30, rank_within_parent#31]
 
-(42) TakeOrderedAndProject
+(43) TakeOrderedAndProject
 Input [5]: [total_sum#26, s_state#20, s_county#21, lochierarchy#27, rank_within_parent#31]
 Arguments: 100, [lochierarchy#27 DESC NULLS LAST, CASE WHEN (lochierarchy#27 = 0) THEN s_state#20 END ASC NULLS FIRST, rank_within_parent#31 ASC NULLS FIRST], [total_sum#26, s_state#20, s_county#21, lochierarchy#27, rank_within_parent#31]
 
 ===== Subqueries =====
 
 Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4
-BroadcastExchange (47)
-+- * Project (46)
-   +- * Filter (45)
-      +- * ColumnarToRow (44)
-         +- Scan parquet spark_catalog.default.date_dim (43)
+BroadcastExchange (48)
++- * Project (47)
+   +- * Filter (46)
+      +- * ColumnarToRow (45)
+         +- Scan parquet spark_catalog.default.date_dim (44)
 
 
-(43) Scan parquet spark_catalog.default.date_dim
+(44) Scan parquet spark_catalog.default.date_dim
 Output [2]: [d_date_sk#5, d_month_seq#32]
 Batched: true
 Location [not included in comparison]/{warehouse_dir}/date_dim]
 PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)]
 ReadSchema: struct<d_date_sk:int,d_month_seq:int>
 
-(44) ColumnarToRow [codegen id : 1]
+(45) ColumnarToRow [codegen id : 1]
 Input [2]: [d_date_sk#5, d_month_seq#32]
 
-(45) Filter [codegen id : 1]
+(46) Filter [codegen id : 1]
 Input [2]: [d_date_sk#5, d_month_seq#32]
 Condition : (((isnotnull(d_month_seq#32) AND (d_month_seq#32 >= 1200)) AND (d_month_seq#32 <= 1211)) AND isnotnull(d_date_sk#5))
 
-(46) Project [codegen id : 1]
+(47) Project [codegen id : 1]
 Output [1]: [d_date_sk#5]
 Input [2]: [d_date_sk#5, d_month_seq#32]
 
-(47) BroadcastExchange
+(48) BroadcastExchange
 Input [1]: [d_date_sk#5]
 Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7]
 
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/simplified.txt
index 83c53852060..f98948a8cf5 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/simplified.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/simplified.txt
@@ -47,28 +47,29 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count
                                                     Filter [ranking]
                                                       InputAdapter
                                                         Window [_w0,s_state]
-                                                          WholeStageCodegen (5)
-                                                            Sort [s_state,_w0]
-                                                              HashAggregate [sum] [sum(UnscaledValue(ss_net_profit)),_w0,s_state,sum]
-                                                                InputAdapter
-                                                                  Exchange [s_state] #6
-                                                                    WholeStageCodegen (4)
-                                                                      HashAggregate [s_state,ss_net_profit] [sum,sum]
-                                                                        Project [ss_net_profit,s_state]
-                                                                          BroadcastHashJoin [ss_store_sk,s_store_sk]
-                                                                            Project [ss_store_sk,ss_net_profit]
-                                                                              BroadcastHashJoin [ss_sold_date_sk,d_date_sk]
-                                                                                Filter [ss_store_sk]
-                                                                                  ColumnarToRow
-                                                                                    InputAdapter
-                                                                                      Scan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk]
-                                                                                        ReusedSubquery [d_date_sk] #1
-                                                                                InputAdapter
-                                                                                  ReusedExchange [d_date_sk] #3
-                                                                            InputAdapter
-                                                                              BroadcastExchange #7
-                                                                                WholeStageCodegen (3)
-                                                                                  Filter [s_store_sk]
+                                                          WindowGroupLimit [s_state,_w0]
+                                                            WholeStageCodegen (5)
+                                                              Sort [s_state,_w0]
+                                                                HashAggregate [sum] [sum(UnscaledValue(ss_net_profit)),_w0,s_state,sum]
+                                                                  InputAdapter
+                                                                    Exchange [s_state] #6
+                                                                      WholeStageCodegen (4)
+                                                                        HashAggregate [s_state,ss_net_profit] [sum,sum]
+                                                                          Project [ss_net_profit,s_state]
+                                                                            BroadcastHashJoin [ss_store_sk,s_store_sk]
+                                                                              Project [ss_store_sk,ss_net_profit]
+                                                                                BroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+                                                                                  Filter [ss_store_sk]
                                                                                     ColumnarToRow
                                                                                       InputAdapter
-                                                                                        Scan parquet spark_catalog.default.store [s_store_sk,s_state]
+                                                                                        Scan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk]
+                                                                                          ReusedSubquery [d_date_sk] #1
+                                                                                  InputAdapter
+                                                                                    ReusedExchange [d_date_sk] #3
+                                                                              InputAdapter
+                                                                                BroadcastExchange #7
+                                                                                  WholeStageCodegen (3)
+                                                                                    Filter [s_store_sk]
+                                                                                      ColumnarToRow
+                                                                                        InputAdapter
+                                                                                          Scan parquet spark_catalog.default.store [s_store_sk,s_state]
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt
index 13bf02371ef..dade1b4f55c 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt
@@ -1,46 +1,47 @@
 == Physical Plan ==
-TakeOrderedAndProject (42)
-+- * Project (41)
-   +- Window (40)
-      +- * Sort (39)
-         +- Exchange (38)
-            +- * HashAggregate (37)
-               +- Exchange (36)
-                  +- * HashAggregate (35)
-                     +- * Expand (34)
-                        +- * Project (33)
-                           +- * BroadcastHashJoin Inner BuildRight (32)
+TakeOrderedAndProject (43)
++- * Project (42)
+   +- Window (41)
+      +- * Sort (40)
+         +- Exchange (39)
+            +- * HashAggregate (38)
+               +- Exchange (37)
+                  +- * HashAggregate (36)
+                     +- * Expand (35)
+                        +- * Project (34)
+                           +- * BroadcastHashJoin Inner BuildRight (33)
                               :- * Project (6)
                               :  +- * BroadcastHashJoin Inner BuildRight (5)
                               :     :- * Filter (3)
                               :     :  +- * ColumnarToRow (2)
                               :     :     +- Scan parquet spark_catalog.default.store_sales (1)
                               :     +- ReusedExchange (4)
-                              +- BroadcastExchange (31)
-                                 +- * BroadcastHashJoin LeftSemi BuildRight (30)
+                              +- BroadcastExchange (32)
+                                 +- * BroadcastHashJoin LeftSemi BuildRight (31)
                                     :- * Filter (9)
                                     :  +- * ColumnarToRow (8)
                                     :     +- Scan parquet spark_catalog.default.store (7)
-                                    +- BroadcastExchange (29)
-                                       +- * Project (28)
-                                          +- * Filter (27)
-                                             +- Window (26)
-                                                +- * Sort (25)
-                                                   +- * HashAggregate (24)
-                                                      +- Exchange (23)
-                                                         +- * HashAggregate (22)
-                                                            +- * Project (21)
-                                                               +- * BroadcastHashJoin Inner BuildRight (20)
-                                                                  :- * Project (18)
-                                                                  :  +- * BroadcastHashJoin Inner BuildRight (17)
-                                                                  :     :- * Filter (12)
-                                                                  :     :  +- * ColumnarToRow (11)
-                                                                  :     :     +- Scan parquet spark_catalog.default.store_sales (10)
-                                                                  :     +- BroadcastExchange (16)
-                                                                  :        +- * Filter (15)
-                                                                  :           +- * ColumnarToRow (14)
-                                                                  :              +- Scan parquet spark_catalog.default.store (13)
-                                                                  +- ReusedExchange (19)
+                                    +- BroadcastExchange (30)
+                                       +- * Project (29)
+                                          +- * Filter (28)
+                                             +- Window (27)
+                                                +- WindowGroupLimit (26)
+                                                   +- * Sort (25)
+                                                      +- * HashAggregate (24)
+                                                         +- Exchange (23)
+                                                            +- * HashAggregate (22)
+                                                               +- * Project (21)
+                                                                  +- * BroadcastHashJoin Inner BuildRight (20)
+                                                                     :- * Project (18)
+                                                                     :  +- * BroadcastHashJoin Inner BuildRight (17)
+                                                                     :     :- * Filter (12)
+                                                                     :     :  +- * ColumnarToRow (11)
+                                                                     :     :     +- Scan parquet spark_catalog.default.store_sales (10)
+                                                                     :     +- BroadcastExchange (16)
+                                                                     :        +- * Filter (15)
+                                                                     :           +- * ColumnarToRow (14)
+                                                                     :              +- Scan parquet spark_catalog.default.store (13)
+                                                                     +- ReusedExchange (19)
 
 
 (1) Scan parquet spark_catalog.default.store_sales
@@ -58,7 +59,7 @@ Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3]
 Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3]
 Condition : isnotnull(ss_store_sk#1)
 
-(4) ReusedExchange [Reuses operator id: 47]
+(4) ReusedExchange [Reuses operator id: 48]
 Output [1]: [d_date_sk#5]
 
 (5) BroadcastHashJoin [codegen id : 8]
@@ -128,7 +129,7 @@ Join condition: None
 Output [3]: [ss_net_profit#10, ss_sold_date_sk#11, s_state#13]
 Input [5]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11, s_store_sk#12, s_state#13]
 
-(19) ReusedExchange [Reuses operator id: 47]
+(19) ReusedExchange [Reuses operator id: 48]
 Output [1]: [d_date_sk#14]
 
 (20) BroadcastHashJoin [codegen id : 4]
@@ -163,113 +164,117 @@ Results [3]: [s_state#13, MakeDecimal(sum(UnscaledValue(ss_net_profit#10))#17,17
 Input [3]: [s_state#13, _w0#18, s_state#13]
 Arguments: [s_state#13 ASC NULLS FIRST, _w0#18 DESC NULLS LAST], false, 0
 
-(26) Window
+(26) WindowGroupLimit
+Input [3]: [s_state#13, _w0#18, s_state#13]
+Arguments: [s_state#13], [_w0#18 DESC NULLS LAST], rank(_w0#18), 5, Final
+
+(27) Window
 Input [3]: [s_state#13, _w0#18, s_state#13]
 Arguments: [rank(_w0#18) windowspecdefinition(s_state#13, _w0#18 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#19], [s_state#13], [_w0#18 DESC NULLS LAST]
 
-(27) Filter [codegen id : 6]
+(28) Filter [codegen id : 6]
 Input [4]: [s_state#13, _w0#18, s_state#13, ranking#19]
 Condition : (ranking#19 <= 5)
 
-(28) Project [codegen id : 6]
+(29) Project [codegen id : 6]
 Output [1]: [s_state#13]
 Input [4]: [s_state#13, _w0#18, s_state#13, ranking#19]
 
-(29) BroadcastExchange
+(30) BroadcastExchange
 Input [1]: [s_state#13]
 Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=3]
 
-(30) BroadcastHashJoin [codegen id : 7]
+(31) BroadcastHashJoin [codegen id : 7]
 Left keys [1]: [s_state#8]
 Right keys [1]: [s_state#13]
 Join type: LeftSemi
 Join condition: None
 
-(31) BroadcastExchange
+(32) BroadcastExchange
 Input [3]: [s_store_sk#6, s_county#7, s_state#8]
 Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4]
 
-(32) BroadcastHashJoin [codegen id : 8]
+(33) BroadcastHashJoin [codegen id : 8]
 Left keys [1]: [ss_store_sk#1]
 Right keys [1]: [s_store_sk#6]
 Join type: Inner
 Join condition: None
 
-(33) Project [codegen id : 8]
+(34) Project [codegen id : 8]
 Output [3]: [ss_net_profit#2, s_state#8, s_county#7]
 Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#6, s_county#7, s_state#8]
 
-(34) Expand [codegen id : 8]
+(35) Expand [codegen id : 8]
 Input [3]: [ss_net_profit#2, s_state#8, s_county#7]
 Arguments: [[ss_net_profit#2, s_state#8, s_county#7, 0], [ss_net_profit#2, s_state#8, null, 1], [ss_net_profit#2, null, null, 3]], [ss_net_profit#2, s_state#20, s_county#21, spark_grouping_id#22]
 
-(35) HashAggregate [codegen id : 8]
+(36) HashAggregate [codegen id : 8]
 Input [4]: [ss_net_profit#2, s_state#20, s_county#21, spark_grouping_id#22]
 Keys [3]: [s_state#20, s_county#21, spark_grouping_id#22]
 Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))]
 Aggregate Attributes [1]: [sum#23]
 Results [4]: [s_state#20, s_county#21, spark_grouping_id#22, sum#24]
 
-(36) Exchange
+(37) Exchange
 Input [4]: [s_state#20, s_county#21, spark_grouping_id#22, sum#24]
 Arguments: hashpartitioning(s_state#20, s_county#21, spark_grouping_id#22, 5), ENSURE_REQUIREMENTS, [plan_id=5]
 
-(37) HashAggregate [codegen id : 9]
+(38) HashAggregate [codegen id : 9]
 Input [4]: [s_state#20, s_county#21, spark_grouping_id#22, sum#24]
 Keys [3]: [s_state#20, s_county#21, spark_grouping_id#22]
 Functions [1]: [sum(UnscaledValue(ss_net_profit#2))]
 Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#25]
 Results [7]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#25,17,2) AS total_sum#26, s_state#20, s_county#21, (cast((shiftright(spark_grouping_id#22, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#22, 0) & 1) as tinyint)) AS lochierarchy#27, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#25,17,2) AS _w0#28, (cast((shiftright(spark_grouping_id#22, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#22, 0) & 1) as tinyint)) AS _w1#29, CASE WHEN (cast((shiftright(spark_gr [...]
 
-(38) Exchange
+(39) Exchange
 Input [7]: [total_sum#26, s_state#20, s_county#21, lochierarchy#27, _w0#28, _w1#29, _w2#30]
 Arguments: hashpartitioning(_w1#29, _w2#30, 5), ENSURE_REQUIREMENTS, [plan_id=6]
 
-(39) Sort [codegen id : 10]
+(40) Sort [codegen id : 10]
 Input [7]: [total_sum#26, s_state#20, s_county#21, lochierarchy#27, _w0#28, _w1#29, _w2#30]
 Arguments: [_w1#29 ASC NULLS FIRST, _w2#30 ASC NULLS FIRST, _w0#28 DESC NULLS LAST], false, 0
 
-(40) Window
+(41) Window
 Input [7]: [total_sum#26, s_state#20, s_county#21, lochierarchy#27, _w0#28, _w1#29, _w2#30]
 Arguments: [rank(_w0#28) windowspecdefinition(_w1#29, _w2#30, _w0#28 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#31], [_w1#29, _w2#30], [_w0#28 DESC NULLS LAST]
 
-(41) Project [codegen id : 11]
+(42) Project [codegen id : 11]
 Output [5]: [total_sum#26, s_state#20, s_county#21, lochierarchy#27, rank_within_parent#31]
 Input [8]: [total_sum#26, s_state#20, s_county#21, lochierarchy#27, _w0#28, _w1#29, _w2#30, rank_within_parent#31]
 
-(42) TakeOrderedAndProject
+(43) TakeOrderedAndProject
 Input [5]: [total_sum#26, s_state#20, s_county#21, lochierarchy#27, rank_within_parent#31]
 Arguments: 100, [lochierarchy#27 DESC NULLS LAST, CASE WHEN (lochierarchy#27 = 0) THEN s_state#20 END ASC NULLS FIRST, rank_within_parent#31 ASC NULLS FIRST], [total_sum#26, s_state#20, s_county#21, lochierarchy#27, rank_within_parent#31]
 
 ===== Subqueries =====
 
 Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4
-BroadcastExchange (47)
-+- * Project (46)
-   +- * Filter (45)
-      +- * ColumnarToRow (44)
-         +- Scan parquet spark_catalog.default.date_dim (43)
+BroadcastExchange (48)
++- * Project (47)
+   +- * Filter (46)
+      +- * ColumnarToRow (45)
+         +- Scan parquet spark_catalog.default.date_dim (44)
 
 
-(43) Scan parquet spark_catalog.default.date_dim
+(44) Scan parquet spark_catalog.default.date_dim
 Output [2]: [d_date_sk#5, d_month_seq#32]
 Batched: true
 Location [not included in comparison]/{warehouse_dir}/date_dim]
 PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)]
 ReadSchema: struct<d_date_sk:int,d_month_seq:int>
 
-(44) ColumnarToRow [codegen id : 1]
+(45) ColumnarToRow [codegen id : 1]
 Input [2]: [d_date_sk#5, d_month_seq#32]
 
-(45) Filter [codegen id : 1]
+(46) Filter [codegen id : 1]
 Input [2]: [d_date_sk#5, d_month_seq#32]
 Condition : (((isnotnull(d_month_seq#32) AND (d_month_seq#32 >= 1200)) AND (d_month_seq#32 <= 1211)) AND isnotnull(d_date_sk#5))
 
-(46) Project [codegen id : 1]
+(47) Project [codegen id : 1]
 Output [1]: [d_date_sk#5]
 Input [2]: [d_date_sk#5, d_month_seq#32]
 
-(47) BroadcastExchange
+(48) BroadcastExchange
 Input [1]: [d_date_sk#5]
 Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7]
 
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/simplified.txt
index fd43f495475..e61fb9ef8d7 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/simplified.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/simplified.txt
@@ -47,28 +47,29 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count
                                                     Filter [ranking]
                                                       InputAdapter
                                                         Window [_w0,s_state]
-                                                          WholeStageCodegen (5)
-                                                            Sort [s_state,_w0]
-                                                              HashAggregate [sum] [sum(UnscaledValue(ss_net_profit)),_w0,s_state,sum]
-                                                                InputAdapter
-                                                                  Exchange [s_state] #6
-                                                                    WholeStageCodegen (4)
-                                                                      HashAggregate [s_state,ss_net_profit] [sum,sum]
-                                                                        Project [ss_net_profit,s_state]
-                                                                          BroadcastHashJoin [ss_sold_date_sk,d_date_sk]
-                                                                            Project [ss_net_profit,ss_sold_date_sk,s_state]
-                                                                              BroadcastHashJoin [ss_store_sk,s_store_sk]
-                                                                                Filter [ss_store_sk]
-                                                                                  ColumnarToRow
-                                                                                    InputAdapter
-                                                                                      Scan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk]
-                                                                                        ReusedSubquery [d_date_sk] #1
-                                                                                InputAdapter
-                                                                                  BroadcastExchange #7
-                                                                                    WholeStageCodegen (2)
-                                                                                      Filter [s_store_sk]
-                                                                                        ColumnarToRow
-                                                                                          InputAdapter
-                                                                                            Scan parquet spark_catalog.default.store [s_store_sk,s_state]
-                                                                            InputAdapter
-                                                                              ReusedExchange [d_date_sk] #3
+                                                          WindowGroupLimit [s_state,_w0]
+                                                            WholeStageCodegen (5)
+                                                              Sort [s_state,_w0]
+                                                                HashAggregate [sum] [sum(UnscaledValue(ss_net_profit)),_w0,s_state,sum]
+                                                                  InputAdapter
+                                                                    Exchange [s_state] #6
+                                                                      WholeStageCodegen (4)
+                                                                        HashAggregate [s_state,ss_net_profit] [sum,sum]
+                                                                          Project [ss_net_profit,s_state]
+                                                                            BroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+                                                                              Project [ss_net_profit,ss_sold_date_sk,s_state]
+                                                                                BroadcastHashJoin [ss_store_sk,s_store_sk]
+                                                                                  Filter [ss_store_sk]
+                                                                                    ColumnarToRow
+                                                                                      InputAdapter
+                                                                                        Scan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk]
+                                                                                          ReusedSubquery [d_date_sk] #1
+                                                                                  InputAdapter
+                                                                                    BroadcastExchange #7
+                                                                                      WholeStageCodegen (2)
+                                                                                        Filter [s_store_sk]
+                                                                                          ColumnarToRow
+                                                                                            InputAdapter
+                                                                                              Scan parquet spark_catalog.default.store [s_store_sk,s_state]
+                                                                              InputAdapter
+                                                                                ReusedExchange [d_date_sk] #3
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.sf100/explain.txt
index e659b002eaa..11eb66f23fc 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.sf100/explain.txt
@@ -1,74 +1,77 @@
 == Physical Plan ==
-TakeOrderedAndProject (70)
-+- * Filter (69)
-   +- Window (68)
-      +- * Sort (67)
-         +- Exchange (66)
-            +- Union (65)
-               :- * HashAggregate (24)
-               :  +- Exchange (23)
-               :     +- * HashAggregate (22)
-               :        +- * Project (21)
-               :           +- * SortMergeJoin Inner (20)
-               :              :- * Sort (14)
-               :              :  +- Exchange (13)
-               :              :     +- * Project (12)
-               :              :        +- * BroadcastHashJoin Inner BuildRight (11)
-               :              :           :- * Project (6)
-               :              :           :  +- * BroadcastHashJoin Inner BuildRight (5)
-               :              :           :     :- * Filter (3)
-               :              :           :     :  +- * ColumnarToRow (2)
-               :              :           :     :     +- Scan parquet spark_catalog.default.store_sales (1)
-               :              :           :     +- ReusedExchange (4)
-               :              :           +- BroadcastExchange (10)
-               :              :              +- * Filter (9)
-               :              :                 +- * ColumnarToRow (8)
-               :              :                    +- Scan parquet spark_catalog.default.store (7)
-               :              +- * Sort (19)
-               :                 +- Exchange (18)
-               :                    +- * Filter (17)
-               :                       +- * ColumnarToRow (16)
-               :                          +- Scan parquet spark_catalog.default.item (15)
-               :- * HashAggregate (29)
-               :  +- Exchange (28)
-               :     +- * HashAggregate (27)
-               :        +- * HashAggregate (26)
-               :           +- ReusedExchange (25)
-               :- * HashAggregate (34)
-               :  +- Exchange (33)
-               :     +- * HashAggregate (32)
-               :        +- * HashAggregate (31)
-               :           +- ReusedExchange (30)
-               :- * HashAggregate (39)
-               :  +- Exchange (38)
-               :     +- * HashAggregate (37)
-               :        +- * HashAggregate (36)
-               :           +- ReusedExchange (35)
-               :- * HashAggregate (44)
-               :  +- Exchange (43)
-               :     +- * HashAggregate (42)
-               :        +- * HashAggregate (41)
-               :           +- ReusedExchange (40)
-               :- * HashAggregate (49)
-               :  +- Exchange (48)
-               :     +- * HashAggregate (47)
-               :        +- * HashAggregate (46)
-               :           +- ReusedExchange (45)
-               :- * HashAggregate (54)
-               :  +- Exchange (53)
-               :     +- * HashAggregate (52)
-               :        +- * HashAggregate (51)
-               :           +- ReusedExchange (50)
-               :- * HashAggregate (59)
-               :  +- Exchange (58)
-               :     +- * HashAggregate (57)
-               :        +- * HashAggregate (56)
-               :           +- ReusedExchange (55)
-               +- * HashAggregate (64)
-                  +- Exchange (63)
-                     +- * HashAggregate (62)
-                        +- * HashAggregate (61)
-                           +- ReusedExchange (60)
+TakeOrderedAndProject (73)
++- * Filter (72)
+   +- Window (71)
+      +- WindowGroupLimit (70)
+         +- * Sort (69)
+            +- Exchange (68)
+               +- WindowGroupLimit (67)
+                  +- * Sort (66)
+                     +- Union (65)
+                        :- * HashAggregate (24)
+                        :  +- Exchange (23)
+                        :     +- * HashAggregate (22)
+                        :        +- * Project (21)
+                        :           +- * SortMergeJoin Inner (20)
+                        :              :- * Sort (14)
+                        :              :  +- Exchange (13)
+                        :              :     +- * Project (12)
+                        :              :        +- * BroadcastHashJoin Inner BuildRight (11)
+                        :              :           :- * Project (6)
+                        :              :           :  +- * BroadcastHashJoin Inner BuildRight (5)
+                        :              :           :     :- * Filter (3)
+                        :              :           :     :  +- * ColumnarToRow (2)
+                        :              :           :     :     +- Scan parquet spark_catalog.default.store_sales (1)
+                        :              :           :     +- ReusedExchange (4)
+                        :              :           +- BroadcastExchange (10)
+                        :              :              +- * Filter (9)
+                        :              :                 +- * ColumnarToRow (8)
+                        :              :                    +- Scan parquet spark_catalog.default.store (7)
+                        :              +- * Sort (19)
+                        :                 +- Exchange (18)
+                        :                    +- * Filter (17)
+                        :                       +- * ColumnarToRow (16)
+                        :                          +- Scan parquet spark_catalog.default.item (15)
+                        :- * HashAggregate (29)
+                        :  +- Exchange (28)
+                        :     +- * HashAggregate (27)
+                        :        +- * HashAggregate (26)
+                        :           +- ReusedExchange (25)
+                        :- * HashAggregate (34)
+                        :  +- Exchange (33)
+                        :     +- * HashAggregate (32)
+                        :        +- * HashAggregate (31)
+                        :           +- ReusedExchange (30)
+                        :- * HashAggregate (39)
+                        :  +- Exchange (38)
+                        :     +- * HashAggregate (37)
+                        :        +- * HashAggregate (36)
+                        :           +- ReusedExchange (35)
+                        :- * HashAggregate (44)
+                        :  +- Exchange (43)
+                        :     +- * HashAggregate (42)
+                        :        +- * HashAggregate (41)
+                        :           +- ReusedExchange (40)
+                        :- * HashAggregate (49)
+                        :  +- Exchange (48)
+                        :     +- * HashAggregate (47)
+                        :        +- * HashAggregate (46)
+                        :           +- ReusedExchange (45)
+                        :- * HashAggregate (54)
+                        :  +- Exchange (53)
+                        :     +- * HashAggregate (52)
+                        :        +- * HashAggregate (51)
+                        :           +- ReusedExchange (50)
+                        :- * HashAggregate (59)
+                        :  +- Exchange (58)
+                        :     +- * HashAggregate (57)
+                        :        +- * HashAggregate (56)
+                        :           +- ReusedExchange (55)
+                        +- * HashAggregate (64)
+                           +- Exchange (63)
+                              +- * HashAggregate (62)
+                                 +- * HashAggregate (61)
+                                    +- ReusedExchange (60)
 
 
 (1) Scan parquet spark_catalog.default.store_sales
@@ -86,7 +89,7 @@ Input [5]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sol
 Input [5]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5]
 Condition : (isnotnull(ss_store_sk#2) AND isnotnull(ss_item_sk#1))
 
-(4) ReusedExchange [Reuses operator id: 75]
+(4) ReusedExchange [Reuses operator id: 78]
 Output [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10]
 
 (5) BroadcastHashJoin [codegen id : 3]
@@ -411,55 +414,67 @@ Results [9]: [null AS i_category#116, null AS i_class#117, null AS i_brand#118,
 
 (65) Union
 
-(66) Exchange
+(66) Sort [codegen id : 81]
+Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23]
+Arguments: [i_category#16 ASC NULLS FIRST, sumsales#23 DESC NULLS LAST], false, 0
+
+(67) WindowGroupLimit
+Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23]
+Arguments: [i_category#16], [sumsales#23 DESC NULLS LAST], rank(sumsales#23), 100, Partial
+
+(68) Exchange
 Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23]
 Arguments: hashpartitioning(i_category#16, 5), ENSURE_REQUIREMENTS, [plan_id=13]
 
-(67) Sort [codegen id : 81]
+(69) Sort [codegen id : 82]
 Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23]
 Arguments: [i_category#16 ASC NULLS FIRST, sumsales#23 DESC NULLS LAST], false, 0
 
-(68) Window
+(70) WindowGroupLimit
+Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23]
+Arguments: [i_category#16], [sumsales#23 DESC NULLS LAST], rank(sumsales#23), 100, Final
+
+(71) Window
 Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23]
 Arguments: [rank(sumsales#23) windowspecdefinition(i_category#16, sumsales#23 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#125], [i_category#16], [sumsales#23 DESC NULLS LAST]
 
-(69) Filter [codegen id : 82]
+(72) Filter [codegen id : 83]
 Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23, rk#125]
 Condition : (rk#125 <= 100)
 
-(70) TakeOrderedAndProject
+(73) TakeOrderedAndProject
 Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23, rk#125]
 Arguments: 100, [i_category#16 ASC NULLS FIRST, i_class#15 ASC NULLS FIRST, i_brand#14 ASC NULLS FIRST, i_product_name#17 ASC NULLS FIRST, d_year#8 ASC NULLS FIRST, d_qoy#10 ASC NULLS FIRST, d_moy#9 ASC NULLS FIRST, s_store_id#12 ASC NULLS FIRST, sumsales#23 ASC NULLS FIRST, rk#125 ASC NULLS FIRST], [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23, rk#125]
 
 ===== Subqueries =====
 
 Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6
-BroadcastExchange (75)
-+- * Project (74)
-   +- * Filter (73)
-      +- * ColumnarToRow (72)
-         +- Scan parquet spark_catalog.default.date_dim (71)
+BroadcastExchange (78)
++- * Project (77)
+   +- * Filter (76)
+      +- * ColumnarToRow (75)
+         +- Scan parquet spark_catalog.default.date_dim (74)
 
 
-(71) Scan parquet spark_catalog.default.date_dim
+(74) Scan parquet spark_catalog.default.date_dim
 Output [5]: [d_date_sk#7, d_month_seq#126, d_year#8, d_moy#9, d_qoy#10]
 Batched: true
 Location [not included in comparison]/{warehouse_dir}/date_dim]
 PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)]
 ReadSchema: struct<d_date_sk:int,d_month_seq:int,d_year:int,d_moy:int,d_qoy:int>
 
-(72) ColumnarToRow [codegen id : 1]
+(75) ColumnarToRow [codegen id : 1]
 Input [5]: [d_date_sk#7, d_month_seq#126, d_year#8, d_moy#9, d_qoy#10]
 
-(73) Filter [codegen id : 1]
+(76) Filter [codegen id : 1]
 Input [5]: [d_date_sk#7, d_month_seq#126, d_year#8, d_moy#9, d_qoy#10]
 Condition : (((isnotnull(d_month_seq#126) AND (d_month_seq#126 >= 1212)) AND (d_month_seq#126 <= 1223)) AND isnotnull(d_date_sk#7))
 
-(74) Project [codegen id : 1]
+(77) Project [codegen id : 1]
 Output [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10]
 Input [5]: [d_date_sk#7, d_month_seq#126, d_year#8, d_moy#9, d_qoy#10]
 
-(75) BroadcastExchange
+(78) BroadcastExchange
 Input [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10]
 Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14]
 
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.sf100/simplified.txt
index 58c8a6f0b11..795fa297b9b 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.sf100/simplified.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.sf100/simplified.txt
@@ -1,131 +1,136 @@
 TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,rk]
-  WholeStageCodegen (82)
+  WholeStageCodegen (83)
     Filter [rk]
       InputAdapter
         Window [sumsales,i_category]
-          WholeStageCodegen (81)
-            Sort [i_category,sumsales]
-              InputAdapter
-                Exchange [i_category] #1
-                  Union
-                    WholeStageCodegen (8)
-                      HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty]
-                        InputAdapter
-                          Exchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] #2
-                            WholeStageCodegen (7)
-                              HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,ss_sales_price,ss_quantity] [sum,isEmpty,sum,isEmpty]
-                                Project [ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_brand,i_class,i_category,i_product_name]
-                                  SortMergeJoin [ss_item_sk,i_item_sk]
-                                    InputAdapter
-                                      WholeStageCodegen (4)
-                                        Sort [ss_item_sk]
-                                          InputAdapter
-                                            Exchange [ss_item_sk] #3
-                                              WholeStageCodegen (3)
-                                                Project [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id]
-                                                  BroadcastHashJoin [ss_store_sk,s_store_sk]
-                                                    Project [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy]
-                                                      BroadcastHashJoin [ss_sold_date_sk,d_date_sk]
-                                                        Filter [ss_store_sk,ss_item_sk]
-                                                          ColumnarToRow
-                                                            InputAdapter
-                                                              Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk]
-                                                                SubqueryBroadcast [d_date_sk] #1
-                                                                  BroadcastExchange #4
-                                                                    WholeStageCodegen (1)
-                                                                      Project [d_date_sk,d_year,d_moy,d_qoy]
-                                                                        Filter [d_month_seq,d_date_sk]
-                                                                          ColumnarToRow
-                                                                            InputAdapter
-                                                                              Scan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy]
-                                                        InputAdapter
-                                                          ReusedExchange [d_date_sk,d_year,d_moy,d_qoy] #4
+          WindowGroupLimit [i_category,sumsales]
+            WholeStageCodegen (82)
+              Sort [i_category,sumsales]
+                InputAdapter
+                  Exchange [i_category] #1
+                    WindowGroupLimit [i_category,sumsales]
+                      WholeStageCodegen (81)
+                        Sort [i_category,sumsales]
+                          InputAdapter
+                            Union
+                              WholeStageCodegen (8)
+                                HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty]
+                                  InputAdapter
+                                    Exchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] #2
+                                      WholeStageCodegen (7)
+                                        HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,ss_sales_price,ss_quantity] [sum,isEmpty,sum,isEmpty]
+                                          Project [ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_brand,i_class,i_category,i_product_name]
+                                            SortMergeJoin [ss_item_sk,i_item_sk]
+                                              InputAdapter
+                                                WholeStageCodegen (4)
+                                                  Sort [ss_item_sk]
                                                     InputAdapter
-                                                      BroadcastExchange #5
-                                                        WholeStageCodegen (2)
-                                                          Filter [s_store_sk]
-                                                            ColumnarToRow
+                                                      Exchange [ss_item_sk] #3
+                                                        WholeStageCodegen (3)
+                                                          Project [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id]
+                                                            BroadcastHashJoin [ss_store_sk,s_store_sk]
+                                                              Project [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy]
+                                                                BroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+                                                                  Filter [ss_store_sk,ss_item_sk]
+                                                                    ColumnarToRow
+                                                                      InputAdapter
+                                                                        Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk]
+                                                                          SubqueryBroadcast [d_date_sk] #1
+                                                                            BroadcastExchange #4
+                                                                              WholeStageCodegen (1)
+                                                                                Project [d_date_sk,d_year,d_moy,d_qoy]
+                                                                                  Filter [d_month_seq,d_date_sk]
+                                                                                    ColumnarToRow
+                                                                                      InputAdapter
+                                                                                        Scan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy]
+                                                                  InputAdapter
+                                                                    ReusedExchange [d_date_sk,d_year,d_moy,d_qoy] #4
                                                               InputAdapter
-                                                                Scan parquet spark_catalog.default.store [s_store_sk,s_store_id]
-                                    InputAdapter
-                                      WholeStageCodegen (6)
-                                        Sort [i_item_sk]
-                                          InputAdapter
-                                            Exchange [i_item_sk] #6
-                                              WholeStageCodegen (5)
-                                                Filter [i_item_sk]
-                                                  ColumnarToRow
+                                                                BroadcastExchange #5
+                                                                  WholeStageCodegen (2)
+                                                                    Filter [s_store_sk]
+                                                                      ColumnarToRow
+                                                                        InputAdapter
+                                                                          Scan parquet spark_catalog.default.store [s_store_sk,s_store_id]
+                                              InputAdapter
+                                                WholeStageCodegen (6)
+                                                  Sort [i_item_sk]
                                                     InputAdapter
-                                                      Scan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name]
-                    WholeStageCodegen (17)
-                      HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sum,isEmpty] [sum(sumsales),s_store_id,sumsales,sum,isEmpty]
-                        InputAdapter
-                          Exchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy] #7
-                            WholeStageCodegen (16)
-                              HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sumsales] [sum,isEmpty,sum,isEmpty]
-                                HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty]
+                                                      Exchange [i_item_sk] #6
+                                                        WholeStageCodegen (5)
+                                                          Filter [i_item_sk]
+                                                            ColumnarToRow
+                                                              InputAdapter
+                                                                Scan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name]
+                              WholeStageCodegen (17)
+                                HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sum,isEmpty] [sum(sumsales),s_store_id,sumsales,sum,isEmpty]
                                   InputAdapter
-                                    ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2
-                    WholeStageCodegen (26)
-                      HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sum,isEmpty] [sum(sumsales),d_moy,s_store_id,sumsales,sum,isEmpty]
-                        InputAdapter
-                          Exchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy] #8
-                            WholeStageCodegen (25)
-                              HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sumsales] [sum,isEmpty,sum,isEmpty]
-                                HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty]
+                                    Exchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy] #7
+                                      WholeStageCodegen (16)
+                                        HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sumsales] [sum,isEmpty,sum,isEmpty]
+                                          HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty]
+                                            InputAdapter
+                                              ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2
+                              WholeStageCodegen (26)
+                                HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sum,isEmpty] [sum(sumsales),d_moy,s_store_id,sumsales,sum,isEmpty]
                                   InputAdapter
-                                    ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2
-                    WholeStageCodegen (35)
-                      HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,sum,isEmpty] [sum(sumsales),d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty]
-                        InputAdapter
-                          Exchange [i_category,i_class,i_brand,i_product_name,d_year] #9
-                            WholeStageCodegen (34)
-                              HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,sumsales] [sum,isEmpty,sum,isEmpty]
-                                HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty]
+                                    Exchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy] #8
+                                      WholeStageCodegen (25)
+                                        HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sumsales] [sum,isEmpty,sum,isEmpty]
+                                          HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty]
+                                            InputAdapter
+                                              ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2
+                              WholeStageCodegen (35)
+                                HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,sum,isEmpty] [sum(sumsales),d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty]
                                   InputAdapter
-                                    ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2
-                    WholeStageCodegen (44)
-                      HashAggregate [i_category,i_class,i_brand,i_product_name,sum,isEmpty] [sum(sumsales),d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty]
-                        InputAdapter
-                          Exchange [i_category,i_class,i_brand,i_product_name] #10
-                            WholeStageCodegen (43)
-                              HashAggregate [i_category,i_class,i_brand,i_product_name,sumsales] [sum,isEmpty,sum,isEmpty]
-                                HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty]
+                                    Exchange [i_category,i_class,i_brand,i_product_name,d_year] #9
+                                      WholeStageCodegen (34)
+                                        HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,sumsales] [sum,isEmpty,sum,isEmpty]
+                                          HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty]
+                                            InputAdapter
+                                              ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2
+                              WholeStageCodegen (44)
+                                HashAggregate [i_category,i_class,i_brand,i_product_name,sum,isEmpty] [sum(sumsales),d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty]
                                   InputAdapter
-                                    ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2
-                    WholeStageCodegen (53)
-                      HashAggregate [i_category,i_class,i_brand,sum,isEmpty] [sum(sumsales),i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty]
-                        InputAdapter
-                          Exchange [i_category,i_class,i_brand] #11
-                            WholeStageCodegen (52)
-                              HashAggregate [i_category,i_class,i_brand,sumsales] [sum,isEmpty,sum,isEmpty]
-                                HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty]
+                                    Exchange [i_category,i_class,i_brand,i_product_name] #10
+                                      WholeStageCodegen (43)
+                                        HashAggregate [i_category,i_class,i_brand,i_product_name,sumsales] [sum,isEmpty,sum,isEmpty]
+                                          HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty]
+                                            InputAdapter
+                                              ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2
+                              WholeStageCodegen (53)
+                                HashAggregate [i_category,i_class,i_brand,sum,isEmpty] [sum(sumsales),i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty]
                                   InputAdapter
-                                    ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2
-                    WholeStageCodegen (62)
-                      HashAggregate [i_category,i_class,sum,isEmpty] [sum(sumsales),i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty]
-                        InputAdapter
-                          Exchange [i_category,i_class] #12
-                            WholeStageCodegen (61)
-                              HashAggregate [i_category,i_class,sumsales] [sum,isEmpty,sum,isEmpty]
-                                HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty]
+                                    Exchange [i_category,i_class,i_brand] #11
+                                      WholeStageCodegen (52)
+                                        HashAggregate [i_category,i_class,i_brand,sumsales] [sum,isEmpty,sum,isEmpty]
+                                          HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty]
+                                            InputAdapter
+                                              ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2
+                              WholeStageCodegen (62)
+                                HashAggregate [i_category,i_class,sum,isEmpty] [sum(sumsales),i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty]
                                   InputAdapter
-                                    ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2
-                    WholeStageCodegen (71)
-                      HashAggregate [i_category,sum,isEmpty] [sum(sumsales),i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty]
-                        InputAdapter
-                          Exchange [i_category] #13
-                            WholeStageCodegen (70)
-                              HashAggregate [i_category,sumsales] [sum,isEmpty,sum,isEmpty]
-                                HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty]
+                                    Exchange [i_category,i_class] #12
+                                      WholeStageCodegen (61)
+                                        HashAggregate [i_category,i_class,sumsales] [sum,isEmpty,sum,isEmpty]
+                                          HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty]
+                                            InputAdapter
+                                              ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2
+                              WholeStageCodegen (71)
+                                HashAggregate [i_category,sum,isEmpty] [sum(sumsales),i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty]
                                   InputAdapter
-                                    ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2
-                    WholeStageCodegen (80)
-                      HashAggregate [sum,isEmpty] [sum(sumsales),i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty]
-                        InputAdapter
-                          Exchange #14
-                            WholeStageCodegen (79)
-                              HashAggregate [sumsales] [sum,isEmpty,sum,isEmpty]
-                                HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty]
+                                    Exchange [i_category] #13
+                                      WholeStageCodegen (70)
+                                        HashAggregate [i_category,sumsales] [sum,isEmpty,sum,isEmpty]
+                                          HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty]
+                                            InputAdapter
+                                              ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2
+                              WholeStageCodegen (80)
+                                HashAggregate [sum,isEmpty] [sum(sumsales),i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty]
                                   InputAdapter
-                                    ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2
+                                    Exchange #14
+                                      WholeStageCodegen (79)
+                                        HashAggregate [sumsales] [sum,isEmpty,sum,isEmpty]
+                                          HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty]
+                                            InputAdapter
+                                              ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/explain.txt
index a822a95c692..0c95b8405c1 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/explain.txt
@@ -1,71 +1,74 @@
 == Physical Plan ==
-TakeOrderedAndProject (67)
-+- * Filter (66)
-   +- Window (65)
-      +- * Sort (64)
-         +- Exchange (63)
-            +- Union (62)
-               :- * HashAggregate (21)
-               :  +- Exchange (20)
-               :     +- * HashAggregate (19)
-               :        +- * Project (18)
-               :           +- * BroadcastHashJoin Inner BuildRight (17)
-               :              :- * Project (12)
-               :              :  +- * BroadcastHashJoin Inner BuildRight (11)
-               :              :     :- * Project (6)
-               :              :     :  +- * BroadcastHashJoin Inner BuildRight (5)
-               :              :     :     :- * Filter (3)
-               :              :     :     :  +- * ColumnarToRow (2)
-               :              :     :     :     +- Scan parquet spark_catalog.default.store_sales (1)
-               :              :     :     +- ReusedExchange (4)
-               :              :     +- BroadcastExchange (10)
-               :              :        +- * Filter (9)
-               :              :           +- * ColumnarToRow (8)
-               :              :              +- Scan parquet spark_catalog.default.store (7)
-               :              +- BroadcastExchange (16)
-               :                 +- * Filter (15)
-               :                    +- * ColumnarToRow (14)
-               :                       +- Scan parquet spark_catalog.default.item (13)
-               :- * HashAggregate (26)
-               :  +- Exchange (25)
-               :     +- * HashAggregate (24)
-               :        +- * HashAggregate (23)
-               :           +- ReusedExchange (22)
-               :- * HashAggregate (31)
-               :  +- Exchange (30)
-               :     +- * HashAggregate (29)
-               :        +- * HashAggregate (28)
-               :           +- ReusedExchange (27)
-               :- * HashAggregate (36)
-               :  +- Exchange (35)
-               :     +- * HashAggregate (34)
-               :        +- * HashAggregate (33)
-               :           +- ReusedExchange (32)
-               :- * HashAggregate (41)
-               :  +- Exchange (40)
-               :     +- * HashAggregate (39)
-               :        +- * HashAggregate (38)
-               :           +- ReusedExchange (37)
-               :- * HashAggregate (46)
-               :  +- Exchange (45)
-               :     +- * HashAggregate (44)
-               :        +- * HashAggregate (43)
-               :           +- ReusedExchange (42)
-               :- * HashAggregate (51)
-               :  +- Exchange (50)
-               :     +- * HashAggregate (49)
-               :        +- * HashAggregate (48)
-               :           +- ReusedExchange (47)
-               :- * HashAggregate (56)
-               :  +- Exchange (55)
-               :     +- * HashAggregate (54)
-               :        +- * HashAggregate (53)
-               :           +- ReusedExchange (52)
-               +- * HashAggregate (61)
-                  +- Exchange (60)
-                     +- * HashAggregate (59)
-                        +- * HashAggregate (58)
-                           +- ReusedExchange (57)
+TakeOrderedAndProject (70)
++- * Filter (69)
+   +- Window (68)
+      +- WindowGroupLimit (67)
+         +- * Sort (66)
+            +- Exchange (65)
+               +- WindowGroupLimit (64)
+                  +- * Sort (63)
+                     +- Union (62)
+                        :- * HashAggregate (21)
+                        :  +- Exchange (20)
+                        :     +- * HashAggregate (19)
+                        :        +- * Project (18)
+                        :           +- * BroadcastHashJoin Inner BuildRight (17)
+                        :              :- * Project (12)
+                        :              :  +- * BroadcastHashJoin Inner BuildRight (11)
+                        :              :     :- * Project (6)
+                        :              :     :  +- * BroadcastHashJoin Inner BuildRight (5)
+                        :              :     :     :- * Filter (3)
+                        :              :     :     :  +- * ColumnarToRow (2)
+                        :              :     :     :     +- Scan parquet spark_catalog.default.store_sales (1)
+                        :              :     :     +- ReusedExchange (4)
+                        :              :     +- BroadcastExchange (10)
+                        :              :        +- * Filter (9)
+                        :              :           +- * ColumnarToRow (8)
+                        :              :              +- Scan parquet spark_catalog.default.store (7)
+                        :              +- BroadcastExchange (16)
+                        :                 +- * Filter (15)
+                        :                    +- * ColumnarToRow (14)
+                        :                       +- Scan parquet spark_catalog.default.item (13)
+                        :- * HashAggregate (26)
+                        :  +- Exchange (25)
+                        :     +- * HashAggregate (24)
+                        :        +- * HashAggregate (23)
+                        :           +- ReusedExchange (22)
+                        :- * HashAggregate (31)
+                        :  +- Exchange (30)
+                        :     +- * HashAggregate (29)
+                        :        +- * HashAggregate (28)
+                        :           +- ReusedExchange (27)
+                        :- * HashAggregate (36)
+                        :  +- Exchange (35)
+                        :     +- * HashAggregate (34)
+                        :        +- * HashAggregate (33)
+                        :           +- ReusedExchange (32)
+                        :- * HashAggregate (41)
+                        :  +- Exchange (40)
+                        :     +- * HashAggregate (39)
+                        :        +- * HashAggregate (38)
+                        :           +- ReusedExchange (37)
+                        :- * HashAggregate (46)
+                        :  +- Exchange (45)
+                        :     +- * HashAggregate (44)
+                        :        +- * HashAggregate (43)
+                        :           +- ReusedExchange (42)
+                        :- * HashAggregate (51)
+                        :  +- Exchange (50)
+                        :     +- * HashAggregate (49)
+                        :        +- * HashAggregate (48)
+                        :           +- ReusedExchange (47)
+                        :- * HashAggregate (56)
+                        :  +- Exchange (55)
+                        :     +- * HashAggregate (54)
+                        :        +- * HashAggregate (53)
+                        :           +- ReusedExchange (52)
+                        +- * HashAggregate (61)
+                           +- Exchange (60)
+                              +- * HashAggregate (59)
+                                 +- * HashAggregate (58)
+                                    +- ReusedExchange (57)
 
 
 (1) Scan parquet spark_catalog.default.store_sales
@@ -83,7 +86,7 @@ Input [5]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sol
 Input [5]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5]
 Condition : (isnotnull(ss_store_sk#2) AND isnotnull(ss_item_sk#1))
 
-(4) ReusedExchange [Reuses operator id: 72]
+(4) ReusedExchange [Reuses operator id: 75]
 Output [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10]
 
 (5) BroadcastHashJoin [codegen id : 4]
@@ -396,55 +399,67 @@ Results [9]: [null AS i_category#116, null AS i_class#117, null AS i_brand#118,
 
 (62) Union
 
-(63) Exchange
+(63) Sort [codegen id : 54]
+Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23]
+Arguments: [i_category#16 ASC NULLS FIRST, sumsales#23 DESC NULLS LAST], false, 0
+
+(64) WindowGroupLimit
+Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23]
+Arguments: [i_category#16], [sumsales#23 DESC NULLS LAST], rank(sumsales#23), 100, Partial
+
+(65) Exchange
 Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23]
 Arguments: hashpartitioning(i_category#16, 5), ENSURE_REQUIREMENTS, [plan_id=12]
 
-(64) Sort [codegen id : 54]
+(66) Sort [codegen id : 55]
 Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23]
 Arguments: [i_category#16 ASC NULLS FIRST, sumsales#23 DESC NULLS LAST], false, 0
 
-(65) Window
+(67) WindowGroupLimit
+Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23]
+Arguments: [i_category#16], [sumsales#23 DESC NULLS LAST], rank(sumsales#23), 100, Final
+
+(68) Window
 Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23]
 Arguments: [rank(sumsales#23) windowspecdefinition(i_category#16, sumsales#23 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#125], [i_category#16], [sumsales#23 DESC NULLS LAST]
 
-(66) Filter [codegen id : 55]
+(69) Filter [codegen id : 56]
 Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23, rk#125]
 Condition : (rk#125 <= 100)
 
-(67) TakeOrderedAndProject
+(70) TakeOrderedAndProject
 Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23, rk#125]
 Arguments: 100, [i_category#16 ASC NULLS FIRST, i_class#15 ASC NULLS FIRST, i_brand#14 ASC NULLS FIRST, i_product_name#17 ASC NULLS FIRST, d_year#8 ASC NULLS FIRST, d_qoy#10 ASC NULLS FIRST, d_moy#9 ASC NULLS FIRST, s_store_id#12 ASC NULLS FIRST, sumsales#23 ASC NULLS FIRST, rk#125 ASC NULLS FIRST], [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23, rk#125]
 
 ===== Subqueries =====
 
 Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6
-BroadcastExchange (72)
-+- * Project (71)
-   +- * Filter (70)
-      +- * ColumnarToRow (69)
-         +- Scan parquet spark_catalog.default.date_dim (68)
+BroadcastExchange (75)
++- * Project (74)
+   +- * Filter (73)
+      +- * ColumnarToRow (72)
+         +- Scan parquet spark_catalog.default.date_dim (71)
 
 
-(68) Scan parquet spark_catalog.default.date_dim
+(71) Scan parquet spark_catalog.default.date_dim
 Output [5]: [d_date_sk#7, d_month_seq#126, d_year#8, d_moy#9, d_qoy#10]
 Batched: true
 Location [not included in comparison]/{warehouse_dir}/date_dim]
 PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)]
 ReadSchema: struct<d_date_sk:int,d_month_seq:int,d_year:int,d_moy:int,d_qoy:int>
 
-(69) ColumnarToRow [codegen id : 1]
+(72) ColumnarToRow [codegen id : 1]
 Input [5]: [d_date_sk#7, d_month_seq#126, d_year#8, d_moy#9, d_qoy#10]
 
-(70) Filter [codegen id : 1]
+(73) Filter [codegen id : 1]
 Input [5]: [d_date_sk#7, d_month_seq#126, d_year#8, d_moy#9, d_qoy#10]
 Condition : (((isnotnull(d_month_seq#126) AND (d_month_seq#126 >= 1212)) AND (d_month_seq#126 <= 1223)) AND isnotnull(d_date_sk#7))
 
-(71) Project [codegen id : 1]
+(74) Project [codegen id : 1]
 Output [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10]
 Input [5]: [d_date_sk#7, d_month_seq#126, d_year#8, d_moy#9, d_qoy#10]
 
-(72) BroadcastExchange
+(75) BroadcastExchange
 Input [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10]
 Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13]
 
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/simplified.txt
index 29d177dd87f..89393f265a4 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/simplified.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/simplified.txt
@@ -1,122 +1,127 @@
 TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,rk]
-  WholeStageCodegen (55)
+  WholeStageCodegen (56)
     Filter [rk]
       InputAdapter
         Window [sumsales,i_category]
-          WholeStageCodegen (54)
-            Sort [i_category,sumsales]
-              InputAdapter
-                Exchange [i_category] #1
-                  Union
-                    WholeStageCodegen (5)
-                      HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty]
-                        InputAdapter
-                          Exchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] #2
-                            WholeStageCodegen (4)
-                              HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,ss_sales_price,ss_quantity] [sum,isEmpty,sum,isEmpty]
-                                Project [ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_brand,i_class,i_category,i_product_name]
-                                  BroadcastHashJoin [ss_item_sk,i_item_sk]
-                                    Project [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id]
-                                      BroadcastHashJoin [ss_store_sk,s_store_sk]
-                                        Project [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy]
-                                          BroadcastHashJoin [ss_sold_date_sk,d_date_sk]
-                                            Filter [ss_store_sk,ss_item_sk]
-                                              ColumnarToRow
-                                                InputAdapter
-                                                  Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk]
-                                                    SubqueryBroadcast [d_date_sk] #1
-                                                      BroadcastExchange #3
-                                                        WholeStageCodegen (1)
-                                                          Project [d_date_sk,d_year,d_moy,d_qoy]
-                                                            Filter [d_month_seq,d_date_sk]
-                                                              ColumnarToRow
-                                                                InputAdapter
-                                                                  Scan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy]
-                                            InputAdapter
-                                              ReusedExchange [d_date_sk,d_year,d_moy,d_qoy] #3
-                                        InputAdapter
-                                          BroadcastExchange #4
-                                            WholeStageCodegen (2)
-                                              Filter [s_store_sk]
-                                                ColumnarToRow
+          WindowGroupLimit [i_category,sumsales]
+            WholeStageCodegen (55)
+              Sort [i_category,sumsales]
+                InputAdapter
+                  Exchange [i_category] #1
+                    WindowGroupLimit [i_category,sumsales]
+                      WholeStageCodegen (54)
+                        Sort [i_category,sumsales]
+                          InputAdapter
+                            Union
+                              WholeStageCodegen (5)
+                                HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty]
+                                  InputAdapter
+                                    Exchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] #2
+                                      WholeStageCodegen (4)
+                                        HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,ss_sales_price,ss_quantity] [sum,isEmpty,sum,isEmpty]
+                                          Project [ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_brand,i_class,i_category,i_product_name]
+                                            BroadcastHashJoin [ss_item_sk,i_item_sk]
+                                              Project [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id]
+                                                BroadcastHashJoin [ss_store_sk,s_store_sk]
+                                                  Project [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy]
+                                                    BroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+                                                      Filter [ss_store_sk,ss_item_sk]
+                                                        ColumnarToRow
+                                                          InputAdapter
+                                                            Scan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk]
+                                                              SubqueryBroadcast [d_date_sk] #1
+                                                                BroadcastExchange #3
+                                                                  WholeStageCodegen (1)
+                                                                    Project [d_date_sk,d_year,d_moy,d_qoy]
+                                                                      Filter [d_month_seq,d_date_sk]
+                                                                        ColumnarToRow
+                                                                          InputAdapter
+                                                                            Scan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy]
+                                                      InputAdapter
+                                                        ReusedExchange [d_date_sk,d_year,d_moy,d_qoy] #3
                                                   InputAdapter
-                                                    Scan parquet spark_catalog.default.store [s_store_sk,s_store_id]
-                                    InputAdapter
-                                      BroadcastExchange #5
-                                        WholeStageCodegen (3)
-                                          Filter [i_item_sk]
-                                            ColumnarToRow
+                                                    BroadcastExchange #4
+                                                      WholeStageCodegen (2)
+                                                        Filter [s_store_sk]
+                                                          ColumnarToRow
+                                                            InputAdapter
+                                                              Scan parquet spark_catalog.default.store [s_store_sk,s_store_id]
                                               InputAdapter
-                                                Scan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name]
-                    WholeStageCodegen (11)
-                      HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sum,isEmpty] [sum(sumsales),s_store_id,sumsales,sum,isEmpty]
-                        InputAdapter
-                          Exchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy] #6
-                            WholeStageCodegen (10)
-                              HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sumsales] [sum,isEmpty,sum,isEmpty]
-                                HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty]
+                                                BroadcastExchange #5
+                                                  WholeStageCodegen (3)
+                                                    Filter [i_item_sk]
+                                                      ColumnarToRow
+                                                        InputAdapter
+                                                          Scan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name]
+                              WholeStageCodegen (11)
+                                HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sum,isEmpty] [sum(sumsales),s_store_id,sumsales,sum,isEmpty]
                                   InputAdapter
-                                    ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2
-                    WholeStageCodegen (17)
-                      HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sum,isEmpty] [sum(sumsales),d_moy,s_store_id,sumsales,sum,isEmpty]
-                        InputAdapter
-                          Exchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy] #7
-                            WholeStageCodegen (16)
-                              HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sumsales] [sum,isEmpty,sum,isEmpty]
-                                HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty]
+                                    Exchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy] #6
+                                      WholeStageCodegen (10)
+                                        HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sumsales] [sum,isEmpty,sum,isEmpty]
+                                          HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty]
+                                            InputAdapter
+                                              ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2
+                              WholeStageCodegen (17)
+                                HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sum,isEmpty] [sum(sumsales),d_moy,s_store_id,sumsales,sum,isEmpty]
                                   InputAdapter
-                                    ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2
-                    WholeStageCodegen (23)
-                      HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,sum,isEmpty] [sum(sumsales),d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty]
-                        InputAdapter
-                          Exchange [i_category,i_class,i_brand,i_product_name,d_year] #8
-                            WholeStageCodegen (22)
-                              HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,sumsales] [sum,isEmpty,sum,isEmpty]
-                                HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty]
+                                    Exchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy] #7
+                                      WholeStageCodegen (16)
+                                        HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sumsales] [sum,isEmpty,sum,isEmpty]
+                                          HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty]
+                                            InputAdapter
+                                              ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2
+                              WholeStageCodegen (23)
+                                HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,sum,isEmpty] [sum(sumsales),d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty]
                                   InputAdapter
-                                    ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2
-                    WholeStageCodegen (29)
-                      HashAggregate [i_category,i_class,i_brand,i_product_name,sum,isEmpty] [sum(sumsales),d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty]
-                        InputAdapter
-                          Exchange [i_category,i_class,i_brand,i_product_name] #9
-                            WholeStageCodegen (28)
-                              HashAggregate [i_category,i_class,i_brand,i_product_name,sumsales] [sum,isEmpty,sum,isEmpty]
-                                HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty]
+                                    Exchange [i_category,i_class,i_brand,i_product_name,d_year] #8
+                                      WholeStageCodegen (22)
+                                        HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,sumsales] [sum,isEmpty,sum,isEmpty]
+                                          HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty]
+                                            InputAdapter
+                                              ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2
+                              WholeStageCodegen (29)
+                                HashAggregate [i_category,i_class,i_brand,i_product_name,sum,isEmpty] [sum(sumsales),d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty]
                                   InputAdapter
-                                    ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2
-                    WholeStageCodegen (35)
-                      HashAggregate [i_category,i_class,i_brand,sum,isEmpty] [sum(sumsales),i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty]
-                        InputAdapter
-                          Exchange [i_category,i_class,i_brand] #10
-                            WholeStageCodegen (34)
-                              HashAggregate [i_category,i_class,i_brand,sumsales] [sum,isEmpty,sum,isEmpty]
-                                HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty]
+                                    Exchange [i_category,i_class,i_brand,i_product_name] #9
+                                      WholeStageCodegen (28)
+                                        HashAggregate [i_category,i_class,i_brand,i_product_name,sumsales] [sum,isEmpty,sum,isEmpty]
+                                          HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty]
+                                            InputAdapter
+                                              ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2
+                              WholeStageCodegen (35)
+                                HashAggregate [i_category,i_class,i_brand,sum,isEmpty] [sum(sumsales),i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty]
                                   InputAdapter
-                                    ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2
-                    WholeStageCodegen (41)
-                      HashAggregate [i_category,i_class,sum,isEmpty] [sum(sumsales),i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty]
-                        InputAdapter
-                          Exchange [i_category,i_class] #11
-                            WholeStageCodegen (40)
-                              HashAggregate [i_category,i_class,sumsales] [sum,isEmpty,sum,isEmpty]
-                                HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty]
+                                    Exchange [i_category,i_class,i_brand] #10
+                                      WholeStageCodegen (34)
+                                        HashAggregate [i_category,i_class,i_brand,sumsales] [sum,isEmpty,sum,isEmpty]
+                                          HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty]
+                                            InputAdapter
+                                              ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2
+                              WholeStageCodegen (41)
+                                HashAggregate [i_category,i_class,sum,isEmpty] [sum(sumsales),i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty]
                                   InputAdapter
-                                    ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2
-                    WholeStageCodegen (47)
-                      HashAggregate [i_category,sum,isEmpty] [sum(sumsales),i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty]
-                        InputAdapter
-                          Exchange [i_category] #12
-                            WholeStageCodegen (46)
-                              HashAggregate [i_category,sumsales] [sum,isEmpty,sum,isEmpty]
-                                HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty]
+                                    Exchange [i_category,i_class] #11
+                                      WholeStageCodegen (40)
+                                        HashAggregate [i_category,i_class,sumsales] [sum,isEmpty,sum,isEmpty]
+                                          HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty]
+                                            InputAdapter
+                                              ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2
+                              WholeStageCodegen (47)
+                                HashAggregate [i_category,sum,isEmpty] [sum(sumsales),i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty]
                                   InputAdapter
-                                    ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2
-                    WholeStageCodegen (53)
-                      HashAggregate [sum,isEmpty] [sum(sumsales),i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty]
-                        InputAdapter
-                          Exchange #13
-                            WholeStageCodegen (52)
-                              HashAggregate [sumsales] [sum,isEmpty,sum,isEmpty]
-                                HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty]
+                                    Exchange [i_category] #12
+                                      WholeStageCodegen (46)
+                                        HashAggregate [i_category,sumsales] [sum,isEmpty,sum,isEmpty]
+                                          HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty]
+                                            InputAdapter
+                                              ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2
+                              WholeStageCodegen (53)
+                                HashAggregate [sum,isEmpty] [sum(sumsales),i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty]
                                   InputAdapter
-                                    ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2
+                                    Exchange #13
+                                      WholeStageCodegen (52)
+                                        HashAggregate [sumsales] [sum,isEmpty,sum,isEmpty]
+                                          HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty]
+                                            InputAdapter
+                                              ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.sf100/explain.txt
index 81419f9e2ae..ec13bd91484 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.sf100/explain.txt
@@ -1,59 +1,60 @@
 == Physical Plan ==
-TakeOrderedAndProject (55)
-+- * Project (54)
-   +- Window (53)
-      +- * Sort (52)
-         +- Exchange (51)
-            +- * HashAggregate (50)
-               +- Exchange (49)
-                  +- * HashAggregate (48)
-                     +- Union (47)
-                        :- * HashAggregate (36)
-                        :  +- Exchange (35)
-                        :     +- * HashAggregate (34)
-                        :        +- * Project (33)
-                        :           +- * BroadcastHashJoin Inner BuildRight (32)
+TakeOrderedAndProject (56)
++- * Project (55)
+   +- Window (54)
+      +- * Sort (53)
+         +- Exchange (52)
+            +- * HashAggregate (51)
+               +- Exchange (50)
+                  +- * HashAggregate (49)
+                     +- Union (48)
+                        :- * HashAggregate (37)
+                        :  +- Exchange (36)
+                        :     +- * HashAggregate (35)
+                        :        +- * Project (34)
+                        :           +- * BroadcastHashJoin Inner BuildRight (33)
                         :              :- * Project (6)
                         :              :  +- * BroadcastHashJoin Inner BuildRight (5)
                         :              :     :- * Filter (3)
                         :              :     :  +- * ColumnarToRow (2)
                         :              :     :     +- Scan parquet spark_catalog.default.store_sales (1)
                         :              :     +- ReusedExchange (4)
-                        :              +- BroadcastExchange (31)
-                        :                 +- * BroadcastHashJoin LeftSemi BuildRight (30)
+                        :              +- BroadcastExchange (32)
+                        :                 +- * BroadcastHashJoin LeftSemi BuildRight (31)
                         :                    :- * Filter (9)
                         :                    :  +- * ColumnarToRow (8)
                         :                    :     +- Scan parquet spark_catalog.default.store (7)
-                        :                    +- BroadcastExchange (29)
-                        :                       +- * Project (28)
-                        :                          +- * Filter (27)
-                        :                             +- Window (26)
-                        :                                +- * Sort (25)
-                        :                                   +- * HashAggregate (24)
-                        :                                      +- Exchange (23)
-                        :                                         +- * HashAggregate (22)
-                        :                                            +- * Project (21)
-                        :                                               +- * BroadcastHashJoin Inner BuildRight (20)
-                        :                                                  :- * Project (15)
-                        :                                                  :  +- * BroadcastHashJoin Inner BuildRight (14)
-                        :                                                  :     :- * Filter (12)
-                        :                                                  :     :  +- * ColumnarToRow (11)
-                        :                                                  :     :     +- Scan parquet spark_catalog.default.store_sales (10)
-                        :                                                  :     +- ReusedExchange (13)
-                        :                                                  +- BroadcastExchange (19)
-                        :                                                     +- * Filter (18)
-                        :                                                        +- * ColumnarToRow (17)
-                        :                                                           +- Scan parquet spark_catalog.default.store (16)
-                        :- * HashAggregate (41)
-                        :  +- Exchange (40)
-                        :     +- * HashAggregate (39)
-                        :        +- * HashAggregate (38)
-                        :           +- ReusedExchange (37)
-                        +- * HashAggregate (46)
-                           +- Exchange (45)
-                              +- * HashAggregate (44)
-                                 +- * HashAggregate (43)
-                                    +- ReusedExchange (42)
+                        :                    +- BroadcastExchange (30)
+                        :                       +- * Project (29)
+                        :                          +- * Filter (28)
+                        :                             +- Window (27)
+                        :                                +- WindowGroupLimit (26)
+                        :                                   +- * Sort (25)
+                        :                                      +- * HashAggregate (24)
+                        :                                         +- Exchange (23)
+                        :                                            +- * HashAggregate (22)
+                        :                                               +- * Project (21)
+                        :                                                  +- * BroadcastHashJoin Inner BuildRight (20)
+                        :                                                     :- * Project (15)
+                        :                                                     :  +- * BroadcastHashJoin Inner BuildRight (14)
+                        :                                                     :     :- * Filter (12)
+                        :                                                     :     :  +- * ColumnarToRow (11)
+                        :                                                     :     :     +- Scan parquet spark_catalog.default.store_sales (10)
+                        :                                                     :     +- ReusedExchange (13)
+                        :                                                     +- BroadcastExchange (19)
+                        :                                                        +- * Filter (18)
+                        :                                                           +- * ColumnarToRow (17)
+                        :                                                              +- Scan parquet spark_catalog.default.store (16)
+                        :- * HashAggregate (42)
+                        :  +- Exchange (41)
+                        :     +- * HashAggregate (40)
+                        :        +- * HashAggregate (39)
+                        :           +- ReusedExchange (38)
+                        +- * HashAggregate (47)
+                           +- Exchange (46)
+                              +- * HashAggregate (45)
+                                 +- * HashAggregate (44)
+                                    +- ReusedExchange (43)
 
 
 (1) Scan parquet spark_catalog.default.store_sales
@@ -71,7 +72,7 @@ Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3]
 Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3]
 Condition : isnotnull(ss_store_sk#1)
 
-(4) ReusedExchange [Reuses operator id: 60]
+(4) ReusedExchange [Reuses operator id: 61]
 Output [1]: [d_date_sk#5]
 
 (5) BroadcastHashJoin [codegen id : 8]
@@ -113,7 +114,7 @@ Input [3]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11]
 Input [3]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11]
 Condition : isnotnull(ss_store_sk#9)
 
-(13) ReusedExchange [Reuses operator id: 60]
+(13) ReusedExchange [Reuses operator id: 61]
 Output [1]: [d_date_sk#12]
 
 (14) BroadcastHashJoin [codegen id : 4]
@@ -176,185 +177,189 @@ Results [3]: [s_state#14, MakeDecimal(sum(UnscaledValue(ss_net_profit#10))#17,17
 Input [3]: [s_state#14, _w0#18, s_state#14]
 Arguments: [s_state#14 ASC NULLS FIRST, _w0#18 DESC NULLS LAST], false, 0
 
-(26) Window
+(26) WindowGroupLimit
+Input [3]: [s_state#14, _w0#18, s_state#14]
+Arguments: [s_state#14], [_w0#18 DESC NULLS LAST], rank(_w0#18), 5, Final
+
+(27) Window
 Input [3]: [s_state#14, _w0#18, s_state#14]
 Arguments: [rank(_w0#18) windowspecdefinition(s_state#14, _w0#18 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#19], [s_state#14], [_w0#18 DESC NULLS LAST]
 
-(27) Filter [codegen id : 6]
+(28) Filter [codegen id : 6]
 Input [4]: [s_state#14, _w0#18, s_state#14, ranking#19]
 Condition : (ranking#19 <= 5)
 
-(28) Project [codegen id : 6]
+(29) Project [codegen id : 6]
 Output [1]: [s_state#14]
 Input [4]: [s_state#14, _w0#18, s_state#14, ranking#19]
 
-(29) BroadcastExchange
+(30) BroadcastExchange
 Input [1]: [s_state#14]
 Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=3]
 
-(30) BroadcastHashJoin [codegen id : 7]
+(31) BroadcastHashJoin [codegen id : 7]
 Left keys [1]: [s_state#8]
 Right keys [1]: [s_state#14]
 Join type: LeftSemi
 Join condition: None
 
-(31) BroadcastExchange
+(32) BroadcastExchange
 Input [3]: [s_store_sk#6, s_county#7, s_state#8]
 Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4]
 
-(32) BroadcastHashJoin [codegen id : 8]
+(33) BroadcastHashJoin [codegen id : 8]
 Left keys [1]: [ss_store_sk#1]
 Right keys [1]: [s_store_sk#6]
 Join type: Inner
 Join condition: None
 
-(33) Project [codegen id : 8]
+(34) Project [codegen id : 8]
 Output [3]: [ss_net_profit#2, s_county#7, s_state#8]
 Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#6, s_county#7, s_state#8]
 
-(34) HashAggregate [codegen id : 8]
+(35) HashAggregate [codegen id : 8]
 Input [3]: [ss_net_profit#2, s_county#7, s_state#8]
 Keys [2]: [s_state#8, s_county#7]
 Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))]
 Aggregate Attributes [1]: [sum#20]
 Results [3]: [s_state#8, s_county#7, sum#21]
 
-(35) Exchange
+(36) Exchange
 Input [3]: [s_state#8, s_county#7, sum#21]
 Arguments: hashpartitioning(s_state#8, s_county#7, 5), ENSURE_REQUIREMENTS, [plan_id=5]
 
-(36) HashAggregate [codegen id : 9]
+(37) HashAggregate [codegen id : 9]
 Input [3]: [s_state#8, s_county#7, sum#21]
 Keys [2]: [s_state#8, s_county#7]
 Functions [1]: [sum(UnscaledValue(ss_net_profit#2))]
 Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#22]
 Results [6]: [cast(MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#22,17,2) as decimal(27,2)) AS total_sum#23, s_state#8, s_county#7, 0 AS g_state#24, 0 AS g_county#25, 0 AS lochierarchy#26]
 
-(37) ReusedExchange [Reuses operator id: 35]
+(38) ReusedExchange [Reuses operator id: 36]
 Output [3]: [s_state#8, s_county#7, sum#27]
 
-(38) HashAggregate [codegen id : 18]
+(39) HashAggregate [codegen id : 18]
 Input [3]: [s_state#8, s_county#7, sum#27]
 Keys [2]: [s_state#8, s_county#7]
 Functions [1]: [sum(UnscaledValue(ss_net_profit#2))]
 Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#22]
 Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#22,17,2) AS total_sum#28, s_state#8]
 
-(39) HashAggregate [codegen id : 18]
+(40) HashAggregate [codegen id : 18]
 Input [2]: [total_sum#28, s_state#8]
 Keys [1]: [s_state#8]
 Functions [1]: [partial_sum(total_sum#28)]
 Aggregate Attributes [2]: [sum#29, isEmpty#30]
 Results [3]: [s_state#8, sum#31, isEmpty#32]
 
-(40) Exchange
+(41) Exchange
 Input [3]: [s_state#8, sum#31, isEmpty#32]
 Arguments: hashpartitioning(s_state#8, 5), ENSURE_REQUIREMENTS, [plan_id=6]
 
-(41) HashAggregate [codegen id : 19]
+(42) HashAggregate [codegen id : 19]
 Input [3]: [s_state#8, sum#31, isEmpty#32]
 Keys [1]: [s_state#8]
 Functions [1]: [sum(total_sum#28)]
 Aggregate Attributes [1]: [sum(total_sum#28)#33]
 Results [6]: [sum(total_sum#28)#33 AS total_sum#34, s_state#8, null AS s_county#35, 0 AS g_state#36, 1 AS g_county#37, 1 AS lochierarchy#38]
 
-(42) ReusedExchange [Reuses operator id: 35]
+(43) ReusedExchange [Reuses operator id: 36]
 Output [3]: [s_state#8, s_county#7, sum#39]
 
-(43) HashAggregate [codegen id : 28]
+(44) HashAggregate [codegen id : 28]
 Input [3]: [s_state#8, s_county#7, sum#39]
 Keys [2]: [s_state#8, s_county#7]
 Functions [1]: [sum(UnscaledValue(ss_net_profit#2))]
 Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#22]
 Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#22,17,2) AS total_sum#28]
 
-(44) HashAggregate [codegen id : 28]
+(45) HashAggregate [codegen id : 28]
 Input [1]: [total_sum#28]
 Keys: []
 Functions [1]: [partial_sum(total_sum#28)]
 Aggregate Attributes [2]: [sum#40, isEmpty#41]
 Results [2]: [sum#42, isEmpty#43]
 
-(45) Exchange
+(46) Exchange
 Input [2]: [sum#42, isEmpty#43]
 Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=7]
 
-(46) HashAggregate [codegen id : 29]
+(47) HashAggregate [codegen id : 29]
 Input [2]: [sum#42, isEmpty#43]
 Keys: []
 Functions [1]: [sum(total_sum#28)]
 Aggregate Attributes [1]: [sum(total_sum#28)#44]
 Results [6]: [sum(total_sum#28)#44 AS total_sum#45, null AS s_state#46, null AS s_county#47, 1 AS g_state#48, 1 AS g_county#49, 2 AS lochierarchy#50]
 
-(47) Union
+(48) Union
 
-(48) HashAggregate [codegen id : 30]
+(49) HashAggregate [codegen id : 30]
 Input [6]: [total_sum#23, s_state#8, s_county#7, g_state#24, g_county#25, lochierarchy#26]
 Keys [6]: [total_sum#23, s_state#8, s_county#7, g_state#24, g_county#25, lochierarchy#26]
 Functions: []
 Aggregate Attributes: []
 Results [6]: [total_sum#23, s_state#8, s_county#7, g_state#24, g_county#25, lochierarchy#26]
 
-(49) Exchange
+(50) Exchange
 Input [6]: [total_sum#23, s_state#8, s_county#7, g_state#24, g_county#25, lochierarchy#26]
 Arguments: hashpartitioning(total_sum#23, s_state#8, s_county#7, g_state#24, g_county#25, lochierarchy#26, 5), ENSURE_REQUIREMENTS, [plan_id=8]
 
-(50) HashAggregate [codegen id : 31]
+(51) HashAggregate [codegen id : 31]
 Input [6]: [total_sum#23, s_state#8, s_county#7, g_state#24, g_county#25, lochierarchy#26]
 Keys [6]: [total_sum#23, s_state#8, s_county#7, g_state#24, g_county#25, lochierarchy#26]
 Functions: []
 Aggregate Attributes: []
 Results [5]: [total_sum#23, s_state#8, s_county#7, lochierarchy#26, CASE WHEN (g_county#25 = 0) THEN s_state#8 END AS _w0#51]
 
-(51) Exchange
+(52) Exchange
 Input [5]: [total_sum#23, s_state#8, s_county#7, lochierarchy#26, _w0#51]
 Arguments: hashpartitioning(lochierarchy#26, _w0#51, 5), ENSURE_REQUIREMENTS, [plan_id=9]
 
-(52) Sort [codegen id : 32]
+(53) Sort [codegen id : 32]
 Input [5]: [total_sum#23, s_state#8, s_county#7, lochierarchy#26, _w0#51]
 Arguments: [lochierarchy#26 ASC NULLS FIRST, _w0#51 ASC NULLS FIRST, total_sum#23 DESC NULLS LAST], false, 0
 
-(53) Window
+(54) Window
 Input [5]: [total_sum#23, s_state#8, s_county#7, lochierarchy#26, _w0#51]
 Arguments: [rank(total_sum#23) windowspecdefinition(lochierarchy#26, _w0#51, total_sum#23 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#52], [lochierarchy#26, _w0#51], [total_sum#23 DESC NULLS LAST]
 
-(54) Project [codegen id : 33]
+(55) Project [codegen id : 33]
 Output [5]: [total_sum#23, s_state#8, s_county#7, lochierarchy#26, rank_within_parent#52]
 Input [6]: [total_sum#23, s_state#8, s_county#7, lochierarchy#26, _w0#51, rank_within_parent#52]
 
-(55) TakeOrderedAndProject
+(56) TakeOrderedAndProject
 Input [5]: [total_sum#23, s_state#8, s_county#7, lochierarchy#26, rank_within_parent#52]
 Arguments: 100, [lochierarchy#26 DESC NULLS LAST, CASE WHEN (lochierarchy#26 = 0) THEN s_state#8 END ASC NULLS FIRST, rank_within_parent#52 ASC NULLS FIRST], [total_sum#23, s_state#8, s_county#7, lochierarchy#26, rank_within_parent#52]
 
 ===== Subqueries =====
 
 Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4
-BroadcastExchange (60)
-+- * Project (59)
-   +- * Filter (58)
-      +- * ColumnarToRow (57)
-         +- Scan parquet spark_catalog.default.date_dim (56)
+BroadcastExchange (61)
++- * Project (60)
+   +- * Filter (59)
+      +- * ColumnarToRow (58)
+         +- Scan parquet spark_catalog.default.date_dim (57)
 
 
-(56) Scan parquet spark_catalog.default.date_dim
+(57) Scan parquet spark_catalog.default.date_dim
 Output [2]: [d_date_sk#5, d_month_seq#53]
 Batched: true
 Location [not included in comparison]/{warehouse_dir}/date_dim]
 PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)]
 ReadSchema: struct<d_date_sk:int,d_month_seq:int>
 
-(57) ColumnarToRow [codegen id : 1]
+(58) ColumnarToRow [codegen id : 1]
 Input [2]: [d_date_sk#5, d_month_seq#53]
 
-(58) Filter [codegen id : 1]
+(59) Filter [codegen id : 1]
 Input [2]: [d_date_sk#5, d_month_seq#53]
 Condition : (((isnotnull(d_month_seq#53) AND (d_month_seq#53 >= 1212)) AND (d_month_seq#53 <= 1223)) AND isnotnull(d_date_sk#5))
 
-(59) Project [codegen id : 1]
+(60) Project [codegen id : 1]
 Output [1]: [d_date_sk#5]
 Input [2]: [d_date_sk#5, d_month_seq#53]
 
-(60) BroadcastExchange
+(61) BroadcastExchange
 Input [1]: [d_date_sk#5]
 Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=10]
 
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.sf100/simplified.txt
index a93225762bf..2626c57dd12 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.sf100/simplified.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.sf100/simplified.txt
@@ -54,31 +54,32 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count
                                                                   Filter [ranking]
                                                                     InputAdapter
                                                                       Window [_w0,s_state]
-                                                                        WholeStageCodegen (5)
-                                                                          Sort [s_state,_w0]
-                                                                            HashAggregate [sum] [sum(UnscaledValue(ss_net_profit)),_w0,s_state,sum]
-                                                                              InputAdapter
-                                                                                Exchange [s_state] #7
-                                                                                  WholeStageCodegen (4)
-                                                                                    HashAggregate [s_state,ss_net_profit] [sum,sum]
-                                                                                      Project [ss_net_profit,s_state]
-                                                                                        BroadcastHashJoin [ss_store_sk,s_store_sk]
-                                                                                          Project [ss_store_sk,ss_net_profit]
-                                                                                            BroadcastHashJoin [ss_sold_date_sk,d_date_sk]
-                                                                                              Filter [ss_store_sk]
-                                                                                                ColumnarToRow
-                                                                                                  InputAdapter
-                                                                                                    Scan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk]
-                                                                                                      ReusedSubquery [d_date_sk] #1
-                                                                                              InputAdapter
-                                                                                                ReusedExchange [d_date_sk] #4
-                                                                                          InputAdapter
-                                                                                            BroadcastExchange #8
-                                                                                              WholeStageCodegen (3)
-                                                                                                Filter [s_store_sk]
+                                                                        WindowGroupLimit [s_state,_w0]
+                                                                          WholeStageCodegen (5)
+                                                                            Sort [s_state,_w0]
+                                                                              HashAggregate [sum] [sum(UnscaledValue(ss_net_profit)),_w0,s_state,sum]
+                                                                                InputAdapter
+                                                                                  Exchange [s_state] #7
+                                                                                    WholeStageCodegen (4)
+                                                                                      HashAggregate [s_state,ss_net_profit] [sum,sum]
+                                                                                        Project [ss_net_profit,s_state]
+                                                                                          BroadcastHashJoin [ss_store_sk,s_store_sk]
+                                                                                            Project [ss_store_sk,ss_net_profit]
+                                                                                              BroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+                                                                                                Filter [ss_store_sk]
                                                                                                   ColumnarToRow
                                                                                                     InputAdapter
-                                                                                                      Scan parquet spark_catalog.default.store [s_store_sk,s_state]
+                                                                                                      Scan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk]
+                                                                                                        ReusedSubquery [d_date_sk] #1
+                                                                                                InputAdapter
+                                                                                                  ReusedExchange [d_date_sk] #4
+                                                                                            InputAdapter
+                                                                                              BroadcastExchange #8
+                                                                                                WholeStageCodegen (3)
+                                                                                                  Filter [s_store_sk]
+                                                                                                    ColumnarToRow
+                                                                                                      InputAdapter
+                                                                                                        Scan parquet spark_catalog.default.store [s_store_sk,s_state]
                                   WholeStageCodegen (19)
                                     HashAggregate [s_state,sum,isEmpty] [sum(total_sum),total_sum,s_county,g_state,g_county,lochierarchy,sum,isEmpty]
                                       InputAdapter
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/explain.txt
index 9f42a31566c..573d9fe01cc 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/explain.txt
@@ -1,59 +1,60 @@
 == Physical Plan ==
-TakeOrderedAndProject (55)
-+- * Project (54)
-   +- Window (53)
-      +- * Sort (52)
-         +- Exchange (51)
-            +- * HashAggregate (50)
-               +- Exchange (49)
-                  +- * HashAggregate (48)
-                     +- Union (47)
-                        :- * HashAggregate (36)
-                        :  +- Exchange (35)
-                        :     +- * HashAggregate (34)
-                        :        +- * Project (33)
-                        :           +- * BroadcastHashJoin Inner BuildRight (32)
+TakeOrderedAndProject (56)
++- * Project (55)
+   +- Window (54)
+      +- * Sort (53)
+         +- Exchange (52)
+            +- * HashAggregate (51)
+               +- Exchange (50)
+                  +- * HashAggregate (49)
+                     +- Union (48)
+                        :- * HashAggregate (37)
+                        :  +- Exchange (36)
+                        :     +- * HashAggregate (35)
+                        :        +- * Project (34)
+                        :           +- * BroadcastHashJoin Inner BuildRight (33)
                         :              :- * Project (6)
                         :              :  +- * BroadcastHashJoin Inner BuildRight (5)
                         :              :     :- * Filter (3)
                         :              :     :  +- * ColumnarToRow (2)
                         :              :     :     +- Scan parquet spark_catalog.default.store_sales (1)
                         :              :     +- ReusedExchange (4)
-                        :              +- BroadcastExchange (31)
-                        :                 +- * BroadcastHashJoin LeftSemi BuildRight (30)
+                        :              +- BroadcastExchange (32)
+                        :                 +- * BroadcastHashJoin LeftSemi BuildRight (31)
                         :                    :- * Filter (9)
                         :                    :  +- * ColumnarToRow (8)
                         :                    :     +- Scan parquet spark_catalog.default.store (7)
-                        :                    +- BroadcastExchange (29)
-                        :                       +- * Project (28)
-                        :                          +- * Filter (27)
-                        :                             +- Window (26)
-                        :                                +- * Sort (25)
-                        :                                   +- * HashAggregate (24)
-                        :                                      +- Exchange (23)
-                        :                                         +- * HashAggregate (22)
-                        :                                            +- * Project (21)
-                        :                                               +- * BroadcastHashJoin Inner BuildRight (20)
-                        :                                                  :- * Project (18)
-                        :                                                  :  +- * BroadcastHashJoin Inner BuildRight (17)
-                        :                                                  :     :- * Filter (12)
-                        :                                                  :     :  +- * ColumnarToRow (11)
-                        :                                                  :     :     +- Scan parquet spark_catalog.default.store_sales (10)
-                        :                                                  :     +- BroadcastExchange (16)
-                        :                                                  :        +- * Filter (15)
-                        :                                                  :           +- * ColumnarToRow (14)
-                        :                                                  :              +- Scan parquet spark_catalog.default.store (13)
-                        :                                                  +- ReusedExchange (19)
-                        :- * HashAggregate (41)
-                        :  +- Exchange (40)
-                        :     +- * HashAggregate (39)
-                        :        +- * HashAggregate (38)
-                        :           +- ReusedExchange (37)
-                        +- * HashAggregate (46)
-                           +- Exchange (45)
-                              +- * HashAggregate (44)
-                                 +- * HashAggregate (43)
-                                    +- ReusedExchange (42)
+                        :                    +- BroadcastExchange (30)
+                        :                       +- * Project (29)
+                        :                          +- * Filter (28)
+                        :                             +- Window (27)
+                        :                                +- WindowGroupLimit (26)
+                        :                                   +- * Sort (25)
+                        :                                      +- * HashAggregate (24)
+                        :                                         +- Exchange (23)
+                        :                                            +- * HashAggregate (22)
+                        :                                               +- * Project (21)
+                        :                                                  +- * BroadcastHashJoin Inner BuildRight (20)
+                        :                                                     :- * Project (18)
+                        :                                                     :  +- * BroadcastHashJoin Inner BuildRight (17)
+                        :                                                     :     :- * Filter (12)
+                        :                                                     :     :  +- * ColumnarToRow (11)
+                        :                                                     :     :     +- Scan parquet spark_catalog.default.store_sales (10)
+                        :                                                     :     +- BroadcastExchange (16)
+                        :                                                     :        +- * Filter (15)
+                        :                                                     :           +- * ColumnarToRow (14)
+                        :                                                     :              +- Scan parquet spark_catalog.default.store (13)
+                        :                                                     +- ReusedExchange (19)
+                        :- * HashAggregate (42)
+                        :  +- Exchange (41)
+                        :     +- * HashAggregate (40)
+                        :        +- * HashAggregate (39)
+                        :           +- ReusedExchange (38)
+                        +- * HashAggregate (47)
+                           +- Exchange (46)
+                              +- * HashAggregate (45)
+                                 +- * HashAggregate (44)
+                                    +- ReusedExchange (43)
 
 
 (1) Scan parquet spark_catalog.default.store_sales
@@ -71,7 +72,7 @@ Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3]
 Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3]
 Condition : isnotnull(ss_store_sk#1)
 
-(4) ReusedExchange [Reuses operator id: 60]
+(4) ReusedExchange [Reuses operator id: 61]
 Output [1]: [d_date_sk#5]
 
 (5) BroadcastHashJoin [codegen id : 8]
@@ -141,7 +142,7 @@ Join condition: None
 Output [3]: [ss_net_profit#10, ss_sold_date_sk#11, s_state#13]
 Input [5]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11, s_store_sk#12, s_state#13]
 
-(19) ReusedExchange [Reuses operator id: 60]
+(19) ReusedExchange [Reuses operator id: 61]
 Output [1]: [d_date_sk#14]
 
 (20) BroadcastHashJoin [codegen id : 4]
@@ -176,185 +177,189 @@ Results [3]: [s_state#13, MakeDecimal(sum(UnscaledValue(ss_net_profit#10))#17,17
 Input [3]: [s_state#13, _w0#18, s_state#13]
 Arguments: [s_state#13 ASC NULLS FIRST, _w0#18 DESC NULLS LAST], false, 0
 
-(26) Window
+(26) WindowGroupLimit
+Input [3]: [s_state#13, _w0#18, s_state#13]
+Arguments: [s_state#13], [_w0#18 DESC NULLS LAST], rank(_w0#18), 5, Final
+
+(27) Window
 Input [3]: [s_state#13, _w0#18, s_state#13]
 Arguments: [rank(_w0#18) windowspecdefinition(s_state#13, _w0#18 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#19], [s_state#13], [_w0#18 DESC NULLS LAST]
 
-(27) Filter [codegen id : 6]
+(28) Filter [codegen id : 6]
 Input [4]: [s_state#13, _w0#18, s_state#13, ranking#19]
 Condition : (ranking#19 <= 5)
 
-(28) Project [codegen id : 6]
+(29) Project [codegen id : 6]
 Output [1]: [s_state#13]
 Input [4]: [s_state#13, _w0#18, s_state#13, ranking#19]
 
-(29) BroadcastExchange
+(30) BroadcastExchange
 Input [1]: [s_state#13]
 Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=3]
 
-(30) BroadcastHashJoin [codegen id : 7]
+(31) BroadcastHashJoin [codegen id : 7]
 Left keys [1]: [s_state#8]
 Right keys [1]: [s_state#13]
 Join type: LeftSemi
 Join condition: None
 
-(31) BroadcastExchange
+(32) BroadcastExchange
 Input [3]: [s_store_sk#6, s_county#7, s_state#8]
 Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4]
 
-(32) BroadcastHashJoin [codegen id : 8]
+(33) BroadcastHashJoin [codegen id : 8]
 Left keys [1]: [ss_store_sk#1]
 Right keys [1]: [s_store_sk#6]
 Join type: Inner
 Join condition: None
 
-(33) Project [codegen id : 8]
+(34) Project [codegen id : 8]
 Output [3]: [ss_net_profit#2, s_county#7, s_state#8]
 Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#6, s_county#7, s_state#8]
 
-(34) HashAggregate [codegen id : 8]
+(35) HashAggregate [codegen id : 8]
 Input [3]: [ss_net_profit#2, s_county#7, s_state#8]
 Keys [2]: [s_state#8, s_county#7]
 Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))]
 Aggregate Attributes [1]: [sum#20]
 Results [3]: [s_state#8, s_county#7, sum#21]
 
-(35) Exchange
+(36) Exchange
 Input [3]: [s_state#8, s_county#7, sum#21]
 Arguments: hashpartitioning(s_state#8, s_county#7, 5), ENSURE_REQUIREMENTS, [plan_id=5]
 
-(36) HashAggregate [codegen id : 9]
+(37) HashAggregate [codegen id : 9]
 Input [3]: [s_state#8, s_county#7, sum#21]
 Keys [2]: [s_state#8, s_county#7]
 Functions [1]: [sum(UnscaledValue(ss_net_profit#2))]
 Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#22]
 Results [6]: [cast(MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#22,17,2) as decimal(27,2)) AS total_sum#23, s_state#8, s_county#7, 0 AS g_state#24, 0 AS g_county#25, 0 AS lochierarchy#26]
 
-(37) ReusedExchange [Reuses operator id: 35]
+(38) ReusedExchange [Reuses operator id: 36]
 Output [3]: [s_state#8, s_county#7, sum#27]
 
-(38) HashAggregate [codegen id : 18]
+(39) HashAggregate [codegen id : 18]
 Input [3]: [s_state#8, s_county#7, sum#27]
 Keys [2]: [s_state#8, s_county#7]
 Functions [1]: [sum(UnscaledValue(ss_net_profit#2))]
 Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#22]
 Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#22,17,2) AS total_sum#28, s_state#8]
 
-(39) HashAggregate [codegen id : 18]
+(40) HashAggregate [codegen id : 18]
 Input [2]: [total_sum#28, s_state#8]
 Keys [1]: [s_state#8]
 Functions [1]: [partial_sum(total_sum#28)]
 Aggregate Attributes [2]: [sum#29, isEmpty#30]
 Results [3]: [s_state#8, sum#31, isEmpty#32]
 
-(40) Exchange
+(41) Exchange
 Input [3]: [s_state#8, sum#31, isEmpty#32]
 Arguments: hashpartitioning(s_state#8, 5), ENSURE_REQUIREMENTS, [plan_id=6]
 
-(41) HashAggregate [codegen id : 19]
+(42) HashAggregate [codegen id : 19]
 Input [3]: [s_state#8, sum#31, isEmpty#32]
 Keys [1]: [s_state#8]
 Functions [1]: [sum(total_sum#28)]
 Aggregate Attributes [1]: [sum(total_sum#28)#33]
 Results [6]: [sum(total_sum#28)#33 AS total_sum#34, s_state#8, null AS s_county#35, 0 AS g_state#36, 1 AS g_county#37, 1 AS lochierarchy#38]
 
-(42) ReusedExchange [Reuses operator id: 35]
+(43) ReusedExchange [Reuses operator id: 36]
 Output [3]: [s_state#8, s_county#7, sum#39]
 
-(43) HashAggregate [codegen id : 28]
+(44) HashAggregate [codegen id : 28]
 Input [3]: [s_state#8, s_county#7, sum#39]
 Keys [2]: [s_state#8, s_county#7]
 Functions [1]: [sum(UnscaledValue(ss_net_profit#2))]
 Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#22]
 Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#22,17,2) AS total_sum#28]
 
-(44) HashAggregate [codegen id : 28]
+(45) HashAggregate [codegen id : 28]
 Input [1]: [total_sum#28]
 Keys: []
 Functions [1]: [partial_sum(total_sum#28)]
 Aggregate Attributes [2]: [sum#40, isEmpty#41]
 Results [2]: [sum#42, isEmpty#43]
 
-(45) Exchange
+(46) Exchange
 Input [2]: [sum#42, isEmpty#43]
 Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=7]
 
-(46) HashAggregate [codegen id : 29]
+(47) HashAggregate [codegen id : 29]
 Input [2]: [sum#42, isEmpty#43]
 Keys: []
 Functions [1]: [sum(total_sum#28)]
 Aggregate Attributes [1]: [sum(total_sum#28)#44]
 Results [6]: [sum(total_sum#28)#44 AS total_sum#45, null AS s_state#46, null AS s_county#47, 1 AS g_state#48, 1 AS g_county#49, 2 AS lochierarchy#50]
 
-(47) Union
+(48) Union
 
-(48) HashAggregate [codegen id : 30]
+(49) HashAggregate [codegen id : 30]
 Input [6]: [total_sum#23, s_state#8, s_county#7, g_state#24, g_county#25, lochierarchy#26]
 Keys [6]: [total_sum#23, s_state#8, s_county#7, g_state#24, g_county#25, lochierarchy#26]
 Functions: []
 Aggregate Attributes: []
 Results [6]: [total_sum#23, s_state#8, s_county#7, g_state#24, g_county#25, lochierarchy#26]
 
-(49) Exchange
+(50) Exchange
 Input [6]: [total_sum#23, s_state#8, s_county#7, g_state#24, g_county#25, lochierarchy#26]
 Arguments: hashpartitioning(total_sum#23, s_state#8, s_county#7, g_state#24, g_county#25, lochierarchy#26, 5), ENSURE_REQUIREMENTS, [plan_id=8]
 
-(50) HashAggregate [codegen id : 31]
+(51) HashAggregate [codegen id : 31]
 Input [6]: [total_sum#23, s_state#8, s_county#7, g_state#24, g_county#25, lochierarchy#26]
 Keys [6]: [total_sum#23, s_state#8, s_county#7, g_state#24, g_county#25, lochierarchy#26]
 Functions: []
 Aggregate Attributes: []
 Results [5]: [total_sum#23, s_state#8, s_county#7, lochierarchy#26, CASE WHEN (g_county#25 = 0) THEN s_state#8 END AS _w0#51]
 
-(51) Exchange
+(52) Exchange
 Input [5]: [total_sum#23, s_state#8, s_county#7, lochierarchy#26, _w0#51]
 Arguments: hashpartitioning(lochierarchy#26, _w0#51, 5), ENSURE_REQUIREMENTS, [plan_id=9]
 
-(52) Sort [codegen id : 32]
+(53) Sort [codegen id : 32]
 Input [5]: [total_sum#23, s_state#8, s_county#7, lochierarchy#26, _w0#51]
 Arguments: [lochierarchy#26 ASC NULLS FIRST, _w0#51 ASC NULLS FIRST, total_sum#23 DESC NULLS LAST], false, 0
 
-(53) Window
+(54) Window
 Input [5]: [total_sum#23, s_state#8, s_county#7, lochierarchy#26, _w0#51]
 Arguments: [rank(total_sum#23) windowspecdefinition(lochierarchy#26, _w0#51, total_sum#23 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#52], [lochierarchy#26, _w0#51], [total_sum#23 DESC NULLS LAST]
 
-(54) Project [codegen id : 33]
+(55) Project [codegen id : 33]
 Output [5]: [total_sum#23, s_state#8, s_county#7, lochierarchy#26, rank_within_parent#52]
 Input [6]: [total_sum#23, s_state#8, s_county#7, lochierarchy#26, _w0#51, rank_within_parent#52]
 
-(55) TakeOrderedAndProject
+(56) TakeOrderedAndProject
 Input [5]: [total_sum#23, s_state#8, s_county#7, lochierarchy#26, rank_within_parent#52]
 Arguments: 100, [lochierarchy#26 DESC NULLS LAST, CASE WHEN (lochierarchy#26 = 0) THEN s_state#8 END ASC NULLS FIRST, rank_within_parent#52 ASC NULLS FIRST], [total_sum#23, s_state#8, s_county#7, lochierarchy#26, rank_within_parent#52]
 
 ===== Subqueries =====
 
 Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4
-BroadcastExchange (60)
-+- * Project (59)
-   +- * Filter (58)
-      +- * ColumnarToRow (57)
-         +- Scan parquet spark_catalog.default.date_dim (56)
+BroadcastExchange (61)
++- * Project (60)
+   +- * Filter (59)
+      +- * ColumnarToRow (58)
+         +- Scan parquet spark_catalog.default.date_dim (57)
 
 
-(56) Scan parquet spark_catalog.default.date_dim
+(57) Scan parquet spark_catalog.default.date_dim
 Output [2]: [d_date_sk#5, d_month_seq#53]
 Batched: true
 Location [not included in comparison]/{warehouse_dir}/date_dim]
 PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)]
 ReadSchema: struct<d_date_sk:int,d_month_seq:int>
 
-(57) ColumnarToRow [codegen id : 1]
+(58) ColumnarToRow [codegen id : 1]
 Input [2]: [d_date_sk#5, d_month_seq#53]
 
-(58) Filter [codegen id : 1]
+(59) Filter [codegen id : 1]
 Input [2]: [d_date_sk#5, d_month_seq#53]
 Condition : (((isnotnull(d_month_seq#53) AND (d_month_seq#53 >= 1212)) AND (d_month_seq#53 <= 1223)) AND isnotnull(d_date_sk#5))
 
-(59) Project [codegen id : 1]
+(60) Project [codegen id : 1]
 Output [1]: [d_date_sk#5]
 Input [2]: [d_date_sk#5, d_month_seq#53]
 
-(60) BroadcastExchange
+(61) BroadcastExchange
 Input [1]: [d_date_sk#5]
 Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=10]
 
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/simplified.txt
index 75b8b6ecf54..8d0f4d8a9a9 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/simplified.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/simplified.txt
@@ -54,31 +54,32 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count
                                                                   Filter [ranking]
                                                                     InputAdapter
                                                                       Window [_w0,s_state]
-                                                                        WholeStageCodegen (5)
-                                                                          Sort [s_state,_w0]
-                                                                            HashAggregate [sum] [sum(UnscaledValue(ss_net_profit)),_w0,s_state,sum]
-                                                                              InputAdapter
-                                                                                Exchange [s_state] #7
-                                                                                  WholeStageCodegen (4)
-                                                                                    HashAggregate [s_state,ss_net_profit] [sum,sum]
-                                                                                      Project [ss_net_profit,s_state]
-                                                                                        BroadcastHashJoin [ss_sold_date_sk,d_date_sk]
-                                                                                          Project [ss_net_profit,ss_sold_date_sk,s_state]
-                                                                                            BroadcastHashJoin [ss_store_sk,s_store_sk]
-                                                                                              Filter [ss_store_sk]
-                                                                                                ColumnarToRow
-                                                                                                  InputAdapter
-                                                                                                    Scan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk]
-                                                                                                      ReusedSubquery [d_date_sk] #1
-                                                                                              InputAdapter
-                                                                                                BroadcastExchange #8
-                                                                                                  WholeStageCodegen (2)
-                                                                                                    Filter [s_store_sk]
-                                                                                                      ColumnarToRow
-                                                                                                        InputAdapter
-                                                                                                          Scan parquet spark_catalog.default.store [s_store_sk,s_state]
-                                                                                          InputAdapter
-                                                                                            ReusedExchange [d_date_sk] #4
+                                                                        WindowGroupLimit [s_state,_w0]
+                                                                          WholeStageCodegen (5)
+                                                                            Sort [s_state,_w0]
+                                                                              HashAggregate [sum] [sum(UnscaledValue(ss_net_profit)),_w0,s_state,sum]
+                                                                                InputAdapter
+                                                                                  Exchange [s_state] #7
+                                                                                    WholeStageCodegen (4)
+                                                                                      HashAggregate [s_state,ss_net_profit] [sum,sum]
+                                                                                        Project [ss_net_profit,s_state]
+                                                                                          BroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+                                                                                            Project [ss_net_profit,ss_sold_date_sk,s_state]
+                                                                                              BroadcastHashJoin [ss_store_sk,s_store_sk]
+                                                                                                Filter [ss_store_sk]
+                                                                                                  ColumnarToRow
+                                                                                                    InputAdapter
+                                                                                                      Scan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk]
+                                                                                                        ReusedSubquery [d_date_sk] #1
+                                                                                                InputAdapter
+                                                                                                  BroadcastExchange #8
+                                                                                                    WholeStageCodegen (2)
+                                                                                                      Filter [s_store_sk]
+                                                                                                        ColumnarToRow
+                                                                                                          InputAdapter
+                                                                                                            Scan parquet spark_catalog.default.store [s_store_sk,s_state]
+                                                                                            InputAdapter
+                                                                                              ReusedExchange [d_date_sk] #4
                                   WholeStageCodegen (19)
                                     HashAggregate [s_state,sum,isEmpty] [sum(total_sum),total_sum,s_county,g_state,g_county,lochierarchy,sum,isEmpty]
                                       InputAdapter
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala
index 990c1e1b2de..b9421f8b13d 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala
@@ -1265,4 +1265,168 @@ class DataFrameWindowFunctionsSuite extends QueryTest
       )
     )
   }
+
+  test("SPARK-37099: Insert window group limit node for top-k computation") {
+
+    val nullStr: String = null
+    val df = Seq(
+      ("a", 0, "c", 1.0),
+      ("a", 1, "x", 2.0),
+      ("a", 2, "y", 3.0),
+      ("a", 3, "z", -1.0),
+      ("a", 4, "", 2.0),
+      ("a", 4, "", 2.0),
+      ("b", 1, "h", Double.NaN),
+      ("b", 1, "n", Double.PositiveInfinity),
+      ("c", 1, "z", -2.0),
+      ("c", 1, "a", -4.0),
+      ("c", 2, nullStr, 5.0)).toDF("key", "value", "order", "value2")
+
+    val window = Window.partitionBy($"key").orderBy($"order".asc_nulls_first)
+    val window2 = Window.partitionBy($"key").orderBy($"order".desc_nulls_first)
+
+    Seq(-1, 100).foreach { threshold =>
+      withSQLConf(SQLConf.WINDOW_GROUP_LIMIT_THRESHOLD.key -> threshold.toString) {
+        Seq($"rn" === 0, $"rn" < 1, $"rn" <= 0).foreach { condition =>
+          checkAnswer(df.withColumn("rn", row_number().over(window)).where(condition),
+            Seq.empty[Row]
+          )
+        }
+
+        Seq($"rn" === 1, $"rn" < 2, $"rn" <= 1).foreach { condition =>
+          checkAnswer(df.withColumn("rn", row_number().over(window)).where(condition),
+            Seq(
+              Row("a", 4, "", 2.0, 1),
+              Row("b", 1, "h", Double.NaN, 1),
+              Row("c", 2, null, 5.0, 1)
+            )
+          )
+
+          checkAnswer(df.withColumn("rn", rank().over(window)).where(condition),
+            Seq(
+              Row("a", 4, "", 2.0, 1),
+              Row("a", 4, "", 2.0, 1),
+              Row("b", 1, "h", Double.NaN, 1),
+              Row("c", 2, null, 5.0, 1)
+            )
+          )
+
+          checkAnswer(df.withColumn("rn", dense_rank().over(window)).where(condition),
+            Seq(
+              Row("a", 4, "", 2.0, 1),
+              Row("a", 4, "", 2.0, 1),
+              Row("b", 1, "h", Double.NaN, 1),
+              Row("c", 2, null, 5.0, 1)
+            )
+          )
+        }
+
+        Seq($"rn" < 3, $"rn" <= 2).foreach { condition =>
+          checkAnswer(df.withColumn("rn", row_number().over(window)).where(condition),
+            Seq(
+              Row("a", 4, "", 2.0, 1),
+              Row("a", 4, "", 2.0, 2),
+              Row("b", 1, "h", Double.NaN, 1),
+              Row("b", 1, "n", Double.PositiveInfinity, 2),
+              Row("c", 1, "a", -4.0, 2),
+              Row("c", 2, null, 5.0, 1)
+            )
+          )
+
+          checkAnswer(df.withColumn("rn", rank().over(window)).where(condition),
+            Seq(
+              Row("a", 4, "", 2.0, 1),
+              Row("a", 4, "", 2.0, 1),
+              Row("b", 1, "h", Double.NaN, 1),
+              Row("b", 1, "n", Double.PositiveInfinity, 2),
+              Row("c", 1, "a", -4.0, 2),
+              Row("c", 2, null, 5.0, 1)
+            )
+          )
+
+          checkAnswer(df.withColumn("rn", dense_rank().over(window)).where(condition),
+            Seq(
+              Row("a", 0, "c", 1.0, 2),
+              Row("a", 4, "", 2.0, 1),
+              Row("a", 4, "", 2.0, 1),
+              Row("b", 1, "h", Double.NaN, 1),
+              Row("b", 1, "n", Double.PositiveInfinity, 2),
+              Row("c", 1, "a", -4.0, 2),
+              Row("c", 2, null, 5.0, 1)
+            )
+          )
+        }
+
+        val condition = $"rn" === 2 && $"value2" > 0.5
+        checkAnswer(df.withColumn("rn", row_number().over(window)).where(condition),
+          Seq(
+            Row("a", 4, "", 2.0, 2),
+            Row("b", 1, "n", Double.PositiveInfinity, 2)
+          )
+        )
+
+        checkAnswer(df.withColumn("rn", rank().over(window)).where(condition),
+          Seq(
+            Row("b", 1, "n", Double.PositiveInfinity, 2)
+          )
+        )
+
+        checkAnswer(df.withColumn("rn", dense_rank().over(window)).where(condition),
+          Seq(
+            Row("a", 0, "c", 1.0, 2),
+            Row("b", 1, "n", Double.PositiveInfinity, 2)
+          )
+        )
+
+        val multipleRowNumbers = df
+          .withColumn("rn", row_number().over(window))
+          .withColumn("rn2", row_number().over(window))
+          .where('rn < 2 && 'rn2 < 3)
+        checkAnswer(multipleRowNumbers,
+          Seq(
+            Row("a", 4, "", 2.0, 1, 1),
+            Row("b", 1, "h", Double.NaN, 1, 1),
+            Row("c", 2, null, 5.0, 1, 1)
+          )
+        )
+
+        val multipleRanks = df
+          .withColumn("rn", rank().over(window))
+          .withColumn("rn2", rank().over(window))
+          .where('rn < 2 && 'rn2 < 3)
+        checkAnswer(multipleRanks,
+          Seq(
+            Row("a", 4, "", 2.0, 1, 1),
+            Row("a", 4, "", 2.0, 1, 1),
+            Row("b", 1, "h", Double.NaN, 1, 1),
+            Row("c", 2, null, 5.0, 1, 1)
+          )
+        )
+
+        val multipleDenseRanks = df
+          .withColumn("rn", dense_rank().over(window))
+          .withColumn("rn2", dense_rank().over(window))
+          .where('rn < 2 && 'rn2 < 3)
+        checkAnswer(multipleDenseRanks,
+          Seq(
+            Row("a", 4, "", 2.0, 1, 1),
+            Row("a", 4, "", 2.0, 1, 1),
+            Row("b", 1, "h", Double.NaN, 1, 1),
+            Row("c", 2, null, 5.0, 1, 1)
+          )
+        )
+
+        val multipleWindows = df
+          .withColumn("rn2", row_number().over(window2))
+          .withColumn("rn", row_number().over(window))
+          .where('rn < 2 && 'rn2 < 3)
+        checkAnswer(multipleWindows,
+          Seq(
+            Row("b", 1, "h", Double.NaN, 2, 1),
+            Row("c", 2, null, 5.0, 1, 1)
+          )
+        )
+      }
+    }
+  }
 }
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantWindowGroupLimitsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantWindowGroupLimitsSuite.scala
new file mode 100644
index 00000000000..46ed8fdfd21
--- /dev/null
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantWindowGroupLimitsSuite.scala
@@ -0,0 +1,79 @@
+/*
+ * 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
+
+import org.apache.spark.sql.{DataFrame, QueryTest}
+import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanHelper, DisableAdaptiveExecutionSuite, EnableAdaptiveExecutionSuite}
+import org.apache.spark.sql.execution.window.WindowGroupLimitExec
+import org.apache.spark.sql.functions.lit
+import org.apache.spark.sql.test.SharedSparkSession
+
+abstract class RemoveRedundantWindowGroupLimitsSuiteBase
+    extends QueryTest
+    with SharedSparkSession
+    with AdaptiveSparkPlanHelper {
+
+  private def checkNumWindowGroupLimits(df: DataFrame, count: Int): Unit = {
+    val plan = df.queryExecution.executedPlan
+    assert(collectWithSubqueries(plan) { case exec: WindowGroupLimitExec => exec }.length == count)
+  }
+
+  private def checkWindowGroupLimits(query: String, count: Int): Unit = {
+    val df = sql(query)
+    checkNumWindowGroupLimits(df, count)
+    val result = df.collect()
+    checkAnswer(df, result)
+  }
+
+  test("remove redundant WindowGroupLimits") {
+    withTempView("t") {
+      spark.range(0, 100).withColumn("value", lit(1)).createOrReplaceTempView("t")
+      val query1 =
+        """
+          |SELECT *
+          |FROM (
+          |    SELECT id, rank() OVER w AS rn
+          |    FROM t
+          |    GROUP BY id
+          |    WINDOW w AS (PARTITION BY id ORDER BY max(value))
+          |)
+          |WHERE rn < 3
+          |""".stripMargin
+      checkWindowGroupLimits(query1, 1)
+
+      val query2 =
+        """
+          |SELECT *
+          |FROM (
+          |    SELECT id, rank() OVER w AS rn
+          |    FROM t
+          |    GROUP BY id
+          |    WINDOW w AS (ORDER BY max(value))
+          |)
+          |WHERE rn < 3
+          |""".stripMargin
+      checkWindowGroupLimits(query2, 2)
+    }
+  }
+}
+
+class RemoveRedundantWindowGroupLimitsSuite extends RemoveRedundantWindowGroupLimitsSuiteBase
+  with DisableAdaptiveExecutionSuite
+
+class RemoveRedundantWindowGroupLimitsSuiteAE extends RemoveRedundantWindowGroupLimitsSuiteBase
+  with EnableAdaptiveExecutionSuite
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/TopKBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/TopKBenchmark.scala
new file mode 100644
index 00000000000..70853ec3129
--- /dev/null
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/TopKBenchmark.scala
@@ -0,0 +1,76 @@
+/*
+ * 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.benchmark
+
+import org.apache.spark.benchmark.Benchmark
+import org.apache.spark.sql.internal.SQLConf.WINDOW_GROUP_LIMIT_THRESHOLD
+
+/**
+ * Benchmark to measure performance for top-k computation.
+ * To run this benchmark:
+ * {{{
+ *   1. without sbt: bin/spark-submit --class <this class>
+ *      --jars <spark core test jar>,<spark catalyst test jar> <spark sql test jar>
+ *   2. build/sbt "sql/test:runMain <this class>"
+ *   3. generate result: SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain <this class>"
+ *      Results will be written to "benchmarks/TopKBenchmark-results.txt".
+ * }}}
+ */
+object TopKBenchmark extends SqlBasedBenchmark {
+
+  override def runBenchmarkSuite(mainArgs: Array[String]): Unit = {
+
+    runBenchmark("Top-K Computation") {
+      withTempPath { dir =>
+        val path = dir.getCanonicalPath + "/topk_benchmark_table"
+        val N = 1024 * 1024 * 20
+        spark.range(0, N, 1, 11)
+          .selectExpr("id as a", "id % 1024 as b")
+          .write.mode("overwrite")
+          .parquet(path)
+
+        def f(rankLikeFunc: String, partition: String): Unit = {
+          spark.read.parquet(path)
+            .selectExpr(s"$rankLikeFunc() OVER($partition ORDER BY a) AS rn", "a", "b")
+            .where("rn > 100 and rn <= 200")
+            .noop()
+        }
+
+        val benchmark = new Benchmark("Benchmark Top-K", N, minNumIters = 10, output = output)
+
+        Seq("ROW_NUMBER", "RANK", "DENSE_RANK").foreach { function =>
+          Seq("", "PARTITION BY b").foreach { partition =>
+            benchmark.addCase(
+              s"$function (PARTITION: $partition, WindowGroupLimit: false)") { _ =>
+              withSQLConf(WINDOW_GROUP_LIMIT_THRESHOLD.key -> "-1") {
+                f(function, partition)
+              }
+            }
+
+            benchmark.addCase(
+              s"$function (PARTITION: $partition, WindowGroupLimit: true)") { _ =>
+              f(function, partition)
+            }
+          }
+        }
+
+        benchmark.run()
+      }
+    }
+  }
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org