diff --git a/docs/sql-ref-ansi-compliance.md b/docs/sql-ref-ansi-compliance.md
index 7d11e93e8680c..67e138fa2e9c2 100644
--- a/docs/sql-ref-ansi-compliance.md
+++ b/docs/sql-ref-ansi-compliance.md
@@ -452,7 +452,7 @@ Below is a list of all the keywords in Spark SQL.
|NULL|reserved|non-reserved|reserved|
|NULLS|non-reserved|non-reserved|non-reserved|
|OF|non-reserved|non-reserved|reserved|
-|OFFSET|non-reserved|non-reserved|reserved|
+|OFFSET|reserved|non-reserved|reserved|
|ON|reserved|strict-non-reserved|reserved|
|ONLY|reserved|non-reserved|reserved|
|OPTION|non-reserved|non-reserved|non-reserved|
diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/expressions/GeneralScalarExpression.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/expressions/GeneralScalarExpression.java
index a00b05e81802e..134c2f36b510c 100644
--- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/expressions/GeneralScalarExpression.java
+++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/expressions/GeneralScalarExpression.java
@@ -106,6 +106,42 @@
*
Since version: 3.3.0
*
*
+ * Name: GREATEST
+ *
+ * - SQL semantic:
GREATEST(expr, ...)
+ * - Since version: 3.4.0
+ *
+ *
+ * Name: LEAST
+ *
+ * - SQL semantic:
LEAST(expr, ...)
+ * - Since version: 3.4.0
+ *
+ *
+ * Name: RAND
+ *
+ * - SQL semantic:
RAND([seed])
+ * - Since version: 3.4.0
+ *
+ *
+ * Name: LOG
+ *
+ * - SQL semantic:
LOG(base, expr)
+ * - Since version: 3.4.0
+ *
+ *
+ * Name: LOG10
+ *
+ * - SQL semantic:
LOG10(expr)
+ * - Since version: 3.4.0
+ *
+ *
+ * Name: LOG2
+ *
+ * - SQL semantic:
LOG2(expr)
+ * - Since version: 3.4.0
+ *
+ *
* Name: LN
*
* - SQL semantic:
LN(expr)
@@ -142,6 +178,120 @@
* - Since version: 3.3.0
*
*
+ * Name: ROUND
+ *
+ * - SQL semantic:
ROUND(expr, [scale])
+ * - Since version: 3.4.0
+ *
+ *
+ * Name: SIN
+ *
+ * - SQL semantic:
SIN(expr)
+ * - Since version: 3.4.0
+ *
+ *
+ * Name: SINH
+ *
+ * - SQL semantic:
SINH(expr)
+ * - Since version: 3.4.0
+ *
+ *
+ * Name: COS
+ *
+ * - SQL semantic:
COS(expr)
+ * - Since version: 3.4.0
+ *
+ *
+ * Name: COSH
+ *
+ * - SQL semantic:
COSH(expr)
+ * - Since version: 3.4.0
+ *
+ *
+ * Name: TAN
+ *
+ * - SQL semantic:
TAN(expr)
+ * - Since version: 3.4.0
+ *
+ *
+ * Name: TANH
+ *
+ * - SQL semantic:
TANH(expr)
+ * - Since version: 3.4.0
+ *
+ *
+ * Name: COT
+ *
+ * - SQL semantic:
COT(expr)
+ * - Since version: 3.4.0
+ *
+ *
+ * Name: ASIN
+ *
+ * - SQL semantic:
ASIN(expr)
+ * - Since version: 3.4.0
+ *
+ *
+ * Name: ASINH
+ *
+ * - SQL semantic:
ASINH(expr)
+ * - Since version: 3.4.0
+ *
+ *
+ * Name: ACOS
+ *
+ * - SQL semantic:
ACOS(expr)
+ * - Since version: 3.4.0
+ *
+ *
+ * Name: ACOSH
+ *
+ * - SQL semantic:
ACOSH(expr)
+ * - Since version: 3.4.0
+ *
+ *
+ * Name: ATAN
+ *
+ * - SQL semantic:
ATAN(expr)
+ * - Since version: 3.4.0
+ *
+ *
+ * Name: ATANH
+ *
+ * - SQL semantic:
ATANH(expr)
+ * - Since version: 3.4.0
+ *
+ *
+ * Name: ATAN2
+ *
+ * - SQL semantic:
ATAN2(exprY, exprX)
+ * - Since version: 3.4.0
+ *
+ *
+ * Name: CBRT
+ *
+ * - SQL semantic:
CBRT(expr)
+ * - Since version: 3.4.0
+ *
+ *
+ * Name: DEGREES
+ *
+ * - SQL semantic:
DEGREES(expr)
+ * - Since version: 3.4.0
+ *
+ *
+ * Name: RADIANS
+ *
+ * - SQL semantic:
RADIANS(expr)
+ * - Since version: 3.4.0
+ *
+ *
+ * Name: SIGN
+ *
+ * - SQL semantic:
SIGN(expr)
+ * - Since version: 3.4.0
+ *
+ *
* Name: WIDTH_BUCKET
*
* - SQL semantic:
WIDTH_BUCKET(expr)
diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/ScanBuilder.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/ScanBuilder.java
index 27ee534d804ff..f5ce604148b18 100644
--- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/ScanBuilder.java
+++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/ScanBuilder.java
@@ -23,7 +23,8 @@
* An interface for building the {@link Scan}. Implementations can mixin SupportsPushDownXYZ
* interfaces to do operator push down, and keep the operator push down result in the returned
* {@link Scan}. When pushing down operators, the push down order is:
- * sample -> filter -> aggregate -> limit -> column pruning.
+ * sample -> filter -> aggregate -> limit/top-n(sort + limit) -> offset ->
+ * column pruning.
*
* @since 3.0.0
*/
diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/SupportsPushDownLimit.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/SupportsPushDownLimit.java
index 035154d08450a..8a725cd7ed7a8 100644
--- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/SupportsPushDownLimit.java
+++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/SupportsPushDownLimit.java
@@ -21,8 +21,8 @@
/**
* A mix-in interface for {@link ScanBuilder}. Data sources can implement this interface to
- * push down LIMIT. Please note that the combination of LIMIT with other operations
- * such as AGGREGATE, GROUP BY, SORT BY, CLUSTER BY, DISTRIBUTE BY, etc. is NOT pushed down.
+ * push down LIMIT. We can push down LIMIT with many other operations if they follow the
+ * operator order we defined in {@link ScanBuilder}'s class doc.
*
* @since 3.3.0
*/
diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/SupportsPushDownOffset.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/SupportsPushDownOffset.java
new file mode 100644
index 0000000000000..ffa2cad371594
--- /dev/null
+++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/SupportsPushDownOffset.java
@@ -0,0 +1,36 @@
+/*
+ * 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.connector.read;
+
+import org.apache.spark.annotation.Evolving;
+
+/**
+ * A mix-in interface for {@link ScanBuilder}. Data sources can implement this interface to
+ * push down OFFSET. We can push down OFFSET with many other operations if they follow the
+ * operator order we defined in {@link ScanBuilder}'s class doc.
+ *
+ * @since 3.4.0
+ */
+@Evolving
+public interface SupportsPushDownOffset extends ScanBuilder {
+
+ /**
+ * Pushes down OFFSET to the data source.
+ */
+ boolean pushOffset(int offset);
+}
diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/SupportsPushDownTopN.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/SupportsPushDownTopN.java
index cba1592c4fa14..83d15ba2296f5 100644
--- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/SupportsPushDownTopN.java
+++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/SupportsPushDownTopN.java
@@ -22,23 +22,22 @@
/**
* A mix-in interface for {@link ScanBuilder}. Data sources can implement this interface to
- * push down top N(query with ORDER BY ... LIMIT n). Please note that the combination of top N
- * with other operations such as AGGREGATE, GROUP BY, CLUSTER BY, DISTRIBUTE BY, etc.
- * is NOT pushed down.
+ * push down top N(query with ORDER BY ... LIMIT n). We can push down top N with many other
+ * operations if they follow the operator order we defined in {@link ScanBuilder}'s class doc.
*
* @since 3.3.0
*/
@Evolving
public interface SupportsPushDownTopN extends ScanBuilder {
- /**
- * Pushes down top N to the data source.
- */
- boolean pushTopN(SortOrder[] orders, int limit);
+ /**
+ * Pushes down top N to the data source.
+ */
+ boolean pushTopN(SortOrder[] orders, int limit);
- /**
- * Whether the top N is partially pushed or not. If it returns true, then Spark will do top N
- * again. This method will only be called when {@link #pushTopN} returns true.
- */
- default boolean isPartiallyPushed() { return true; }
+ /**
+ * Whether the top N is partially pushed or not. If it returns true, then Spark will do top N
+ * again. This method will only be called when {@link #pushTopN} returns true.
+ */
+ default boolean isPartiallyPushed() { return true; }
}
diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/util/V2ExpressionSQLBuilder.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/util/V2ExpressionSQLBuilder.java
index 396b1d9cdd034..43095807d95dd 100644
--- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/util/V2ExpressionSQLBuilder.java
+++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/util/V2ExpressionSQLBuilder.java
@@ -95,12 +95,37 @@ public String build(Expression expr) {
return visitUnaryArithmetic(name, inputToSQL(e.children()[0]));
case "ABS":
case "COALESCE":
+ case "GREATEST":
+ case "LEAST":
+ case "RAND":
+ case "LOG":
+ case "LOG10":
+ case "LOG2":
case "LN":
case "EXP":
case "POWER":
case "SQRT":
case "FLOOR":
case "CEIL":
+ case "ROUND":
+ case "SIN":
+ case "SINH":
+ case "COS":
+ case "COSH":
+ case "TAN":
+ case "TANH":
+ case "COT":
+ case "ASIN":
+ case "ASINH":
+ case "ACOS":
+ case "ACOSH":
+ case "ATAN":
+ case "ATANH":
+ case "ATAN2":
+ case "CBRT":
+ case "DEGREES":
+ case "RADIANS":
+ case "SIGN":
case "WIDTH_BUCKET":
case "SUBSTRING":
case "UPPER":
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala
index 14c44b0364f31..522eb07c096ee 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala
@@ -393,20 +393,17 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog {
val offset = offsetExpr.eval().asInstanceOf[Int]
if (Int.MaxValue - limit < offset) {
failAnalysis(
- s"""The sum of limit and offset must not be greater than Int.MaxValue,
- | but found limit = $limit, offset = $offset.""".stripMargin)
+ s"""
+ |The sum of the LIMIT clause and the OFFSET clause must not be greater than
+ |the maximum 32-bit integer value (2,147,483,647),
+ |but found limit = $limit, offset = $offset.
+ |""".stripMargin.replace("\n", " "))
}
case _ =>
}
case Offset(offsetExpr, _) => checkLimitLikeClause("offset", offsetExpr)
- case o if !o.isInstanceOf[GlobalLimit] && !o.isInstanceOf[LocalLimit]
- && o.children.exists(_.isInstanceOf[Offset]) =>
- failAnalysis(
- s"""Only the OFFSET clause is allowed in the LIMIT clause, but the OFFSET
- | clause found in: ${o.nodeName}.""".stripMargin)
-
case Tail(limitExpr, _) => checkLimitLikeClause("tail", limitExpr)
case _: Union | _: SetOperation if operator.children.length > 1 =>
@@ -567,7 +564,6 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog {
}
}
checkCollectedMetrics(plan)
- checkOutermostOffset(plan)
extendedCheckRules.foreach(_(plan))
plan.foreachUp {
case o if !o.resolved =>
@@ -578,20 +574,6 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog {
plan.setAnalyzed()
}
- /**
- * Validate that the root node of query or subquery is [[Offset]].
- */
- private def checkOutermostOffset(plan: LogicalPlan): Unit = {
- plan match {
- case Offset(offsetExpr, _) =>
- checkLimitLikeClause("limit", offsetExpr)
- failAnalysis(
- s"""Only the OFFSET clause is allowed in the LIMIT clause, but the OFFSET
- | clause is found to be the outermost node.""".stripMargin)
- case _ =>
- }
- }
-
/**
* Validates subquery expressions in the plan. Upon failure, returns an user facing error.
*/
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
index 1a57ee83fa3ef..0a53e9d73cd58 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
@@ -93,7 +93,7 @@ abstract class Optimizer(catalogManager: CatalogManager)
OptimizeWindowFunctions,
CollapseWindow,
CombineFilters,
- RewriteOffsets,
+ EliminateOffsets,
EliminateLimits,
CombineUnions,
// Constant folding and strength reduction
@@ -639,7 +639,7 @@ object RemoveNoopUnion extends Rule[LogicalPlan] {
}
/**
- * Pushes down [[LocalLimit]] beneath UNION ALL and joins.
+ * Pushes down [[LocalLimit]] beneath UNION ALL, OFFSET and joins.
*/
object LimitPushDown extends Rule[LogicalPlan] {
@@ -709,6 +709,14 @@ object LimitPushDown extends Rule[LogicalPlan] {
// There is a Project between LocalLimit and Join if they do not have the same output.
case LocalLimit(exp, project @ Project(_, join: Join)) =>
LocalLimit(exp, project.copy(child = pushLocalLimitThroughJoin(exp, join)))
+ // Push down limit 1 through Aggregate and turn Aggregate into Project if it is group only.
+ case Limit(le @ IntegerLiteral(1), a: Aggregate) if a.groupOnly =>
+ Limit(le, Project(a.aggregateExpressions, LocalLimit(le, a.child)))
+ case Limit(le @ IntegerLiteral(1), p @ Project(_, a: Aggregate)) if a.groupOnly =>
+ Limit(le, p.copy(child = Project(a.aggregateExpressions, LocalLimit(le, a.child))))
+ // Merge offset value and limit value into LocalLimit and pushes down LocalLimit through Offset.
+ case LocalLimit(le, Offset(oe, grandChild)) =>
+ Offset(oe, LocalLimit(Add(le, oe), grandChild))
}
}
@@ -1784,15 +1792,22 @@ object EliminateLimits extends Rule[LogicalPlan] {
}
/**
- * Rewrite [[Offset]] as [[Limit]] or combines two adjacent [[Offset]] operators into one,
- * merging the expressions into one single expression.
+ * This rule optimizes Offset operators by:
+ * 1. Eliminate [[Offset]] operators if offset == 0.
+ * 2. Replace [[Offset]] operators to empty [[LocalRelation]]
+ * if [[Offset]]'s child max row <= offset.
+ * 3. Combines two adjacent [[Offset]] operators into one, merging the
+ * expressions into one single expression.
*/
-object RewriteOffsets extends Rule[LogicalPlan] {
+object EliminateOffsets extends Rule[LogicalPlan] {
def apply(plan: LogicalPlan): LogicalPlan = plan transform {
- case GlobalLimit(le, Offset(oe, grandChild)) =>
- GlobalLimitAndOffset(le, oe, grandChild)
- case LocalLimit(le, Offset(oe, grandChild)) =>
- Offset(oe, LocalLimit(Add(le, oe), grandChild))
+ case Offset(oe, child) if oe.foldable && oe.eval().asInstanceOf[Int] == 0 =>
+ child
+ case Offset(oe, child)
+ if oe.foldable && child.maxRows.exists(_ <= oe.eval().asInstanceOf[Int]) =>
+ LocalRelation(child.output, data = Seq.empty, isStreaming = child.isStreaming)
+ case Offset(oe1, Offset(oe2, child)) =>
+ Offset(Add(oe1, oe2), child)
}
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PropagateEmptyRelation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PropagateEmptyRelation.scala
index 1a5a8c0fba45d..5b565793fac42 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PropagateEmptyRelation.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PropagateEmptyRelation.scala
@@ -102,7 +102,7 @@ abstract class PropagateEmptyRelationBase extends Rule[LogicalPlan] with CastSup
case _: Sort => empty(p)
case _: GlobalLimit if !p.isStreaming => empty(p)
case _: LocalLimit if !p.isStreaming => empty(p)
- case _: Offset if !p.isStreaming => empty(p)
+ case _: Offset => empty(p)
case _: Repartition => empty(p)
case _: RepartitionByExpression => empty(p)
// An aggregate with non-empty group expression will return one output row per group when the
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala
index d3e6f285fa605..74f643ede4a9f 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala
@@ -915,6 +915,7 @@ object FoldablePropagation extends Rule[LogicalPlan] {
case _: Sample => true
case _: GlobalLimit => true
case _: LocalLimit => true
+ case _: Offset => true
case _: Generate => true
case _: Distinct => true
case _: AppendColumns => true
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 29610c3dcea4c..f16e724e5e4ce 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
@@ -1129,31 +1129,10 @@ case class Offset(offsetExpr: Expression, child: LogicalPlan) extends OrderPrese
case _ => None
}
}
-
override protected def withNewChildInternal(newChild: LogicalPlan): Offset =
copy(child = newChild)
}
-/**
- * A global (coordinated) limit with offset. This operator can skip at most `offsetExpr` number and
- * emit at most `limitExpr` number in total.
- */
-case class GlobalLimitAndOffset(
- limitExpr: Expression,
- offsetExpr: Expression,
- child: LogicalPlan) extends OrderPreservingUnaryNode {
- override def output: Seq[Attribute] = child.output
- override def maxRows: Option[Long] = {
- limitExpr match {
- case IntegerLiteral(limit) => Some(limit)
- case _ => None
- }
- }
-
- override protected def withNewChildInternal(newChild: LogicalPlan): GlobalLimitAndOffset =
- copy(child = newChild)
-}
-
/**
* A constructor for creating a pivot, which will later be converted to a [[Project]]
* or an [[Aggregate]] during the query analysis.
@@ -1265,6 +1244,30 @@ case class LocalLimit(limitExpr: Expression, child: LogicalPlan) extends OrderPr
copy(child = newChild)
}
+object OffsetAndLimit {
+ def unapply(p: GlobalLimit): Option[(Int, Int, LogicalPlan)] = {
+ p match {
+ // Optimizer pushes local limit through offset, so we need to match the plan this way.
+ case GlobalLimit(IntegerLiteral(globalLimit),
+ Offset(IntegerLiteral(offset),
+ LocalLimit(IntegerLiteral(localLimit), child)))
+ if globalLimit + offset == localLimit =>
+ Some((offset, globalLimit, child))
+ case _ => None
+ }
+ }
+}
+
+object LimitAndOffset {
+ def unapply(p: Offset): Option[(Int, Int, LogicalPlan)] = {
+ p match {
+ case Offset(IntegerLiteral(offset), Limit(IntegerLiteral(limit), child)) =>
+ Some((limit, offset, child))
+ case _ => None
+ }
+ }
+}
+
/**
* This is similar with [[Limit]] except:
*
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/SizeInBytesOnlyStatsPlanVisitor.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/SizeInBytesOnlyStatsPlanVisitor.scala
index 3d115fe77176a..d97bb5be36fb1 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/SizeInBytesOnlyStatsPlanVisitor.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/SizeInBytesOnlyStatsPlanVisitor.scala
@@ -93,7 +93,7 @@ object SizeInBytesOnlyStatsPlanVisitor extends LogicalPlanVisitor[Statistics] {
override def visitOffset(p: Offset): Statistics = {
val offset = p.offsetExpr.eval().asInstanceOf[Int]
val childStats = p.child.stats
- val rowCount: BigInt = childStats.rowCount.map(_.-(offset).max(0)).getOrElse(0)
+ val rowCount: BigInt = childStats.rowCount.map(c => c - offset).map(_.max(0)).getOrElse(0)
Statistics(
sizeInBytes = EstimationUtils.getOutputSize(p.output, rowCount, childStats.attributeStats),
rowCount = Some(rowCount))
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala
index 54cdd91c93ea5..9d546b69d0fe6 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala
@@ -30,6 +30,7 @@ import org.apache.spark.sql.catalyst.plans.{Cross, LeftOuter, RightOuter}
import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, GenericArrayData, MapData}
import org.apache.spark.sql.types._
+import org.apache.spark.unsafe.types.UTF8String
private[sql] case class GroupableData(data: Int) {
def getData: Int = data
@@ -549,6 +550,38 @@ class AnalysisErrorSuite extends AnalysisTest {
"The limit expression must be equal to or greater than 0, but got -1" :: Nil
)
+ errorTest(
+ "an evaluated offset class must not be string",
+ testRelation.offset(Literal(UTF8String.fromString("abc"), StringType)),
+ "The offset expression must be integer type, but got string" :: Nil
+ )
+
+ errorTest(
+ "an evaluated offset class must not be long",
+ testRelation.offset(Literal(10L, LongType)),
+ "The offset expression must be integer type, but got bigint" :: Nil
+ )
+
+ errorTest(
+ "an evaluated offset class must not be null",
+ testRelation.offset(Literal(null, IntegerType)),
+ "The evaluated offset expression must not be null, but got " :: Nil
+ )
+
+ errorTest(
+ "num_rows in offset clause must be equal to or greater than 0",
+ testRelation.offset(-1),
+ "The offset expression must be equal to or greater than 0, but got -1" :: Nil
+ )
+
+ errorTest(
+ "the sum of num_rows in limit clause and num_rows in offset clause less than Int.MaxValue",
+ testRelation.offset(Literal(2000000000, IntegerType)).limit(Literal(1000000000, IntegerType)),
+ "The sum of the LIMIT clause and the OFFSET clause must not be greater than" +
+ " the maximum 32-bit integer value (2,147,483,647)," +
+ " but found limit = 1000000000, offset = 2000000000." :: Nil
+ )
+
errorTest(
"more than one generators in SELECT",
listRelation.select(Explode($"list"), Explode($"list")),
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/EliminateOffsetsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/EliminateOffsetsSuite.scala
new file mode 100644
index 0000000000000..d8c0199ac37dc
--- /dev/null
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/EliminateOffsetsSuite.scala
@@ -0,0 +1,100 @@
+/*
+ * 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.Row
+import org.apache.spark.sql.catalyst.dsl.expressions._
+import org.apache.spark.sql.catalyst.dsl.plans._
+import org.apache.spark.sql.catalyst.expressions.{Add, Literal}
+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
+
+class EliminateOffsetsSuite extends PlanTest {
+ object Optimize extends RuleExecutor[LogicalPlan] {
+ val batches =
+ Batch("Eliminate Offset", FixedPoint(10), EliminateOffsets) :: Nil
+ }
+
+ val testRelation = LocalRelation.fromExternalRows(
+ Seq("a".attr.int, "b".attr.int, "c".attr.int),
+ 1.to(10).map(_ => Row(1, 2, 3))
+ )
+
+ test("Offsets: eliminate Offset operators if offset == 0") {
+ val originalQuery =
+ testRelation
+ .select($"a")
+ .offset(0)
+
+ val optimized = Optimize.execute(originalQuery.analyze)
+ val correctAnswer =
+ testRelation
+ .select($"a")
+ .analyze
+
+ comparePlans(optimized, correctAnswer)
+ }
+
+ test("Offsets: cannot eliminate Offset operators if offset > 0") {
+ val originalQuery =
+ testRelation
+ .select($"a")
+ .offset(2)
+
+ val optimized = Optimize.execute(originalQuery.analyze)
+ val correctAnswer =
+ testRelation
+ .select($"a")
+ .offset(2)
+ .analyze
+
+ comparePlans(optimized, correctAnswer)
+ }
+
+ test("Replace Offset operators to empty LocalRelation if child max row <= offset") {
+ val child = testRelation.select($"a").analyze
+ val originalQuery = child.offset(10)
+
+ val optimized = Optimize.execute(originalQuery.analyze)
+ val correctAnswer =
+ LocalRelation(child.output, data = Seq.empty, isStreaming = child.isStreaming).analyze
+
+ comparePlans(optimized, correctAnswer)
+ }
+
+ test("Cannot replace Offset operators to empty LocalRelation if child max row > offset") {
+ val child = testRelation.select($"a").analyze
+ val originalQuery = child.offset(3)
+
+ val optimized = Optimize.execute(originalQuery.analyze)
+ val correctAnswer = originalQuery.analyze
+
+ comparePlans(optimized, correctAnswer)
+ }
+
+ test("Combines Offset operators") {
+ val child = testRelation.select($"a").analyze
+ val originalQuery = child.offset(2).offset(3)
+
+ val optimized = Optimize.execute(originalQuery.analyze)
+ val correctAnswer = child.offset(Add(Literal(3), Literal(2))).analyze
+
+ comparePlans(optimized, correctAnswer)
+ }
+}
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LimitPushdownSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LimitPushdownSuite.scala
index c2503e362c8c8..472871a813c52 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LimitPushdownSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LimitPushdownSuite.scala
@@ -239,4 +239,10 @@ class LimitPushdownSuite extends PlanTest {
Limit(5, LocalLimit(5, x).join(y, LeftOuter, joinCondition).select("x.a".attr)).analyze
comparePlans(optimized, correctAnswer)
}
+
+ test("Push down limit 1 through Offset") {
+ comparePlans(
+ Optimize.execute(testRelation.offset(2).limit(1).analyze),
+ GlobalLimit(1, Offset(2, LocalLimit(3, testRelation))).analyze)
+ }
}
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/BasicStatsEstimationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/BasicStatsEstimationSuite.scala
index 7fa9a3141d06a..1097dafcafedc 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/BasicStatsEstimationSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/BasicStatsEstimationSuite.scala
@@ -185,6 +185,22 @@ class BasicStatsEstimationSuite extends PlanTest with StatsEstimationTestBase {
expectedStatsCboOff = windowsStats)
}
+ test("offset estimation: offset < child's rowCount") {
+ val offset = Offset(Literal(2), plan)
+ checkStats(offset, Statistics(sizeInBytes = 96, rowCount = Some(8)))
+ }
+
+ test("offset estimation: offset > child's rowCount") {
+ val offset = Offset(Literal(20), plan)
+ checkStats(offset, Statistics(sizeInBytes = 1, rowCount = Some(0)))
+ }
+
+ test("offset estimation: offset = 0") {
+ val offset = Offset(Literal(0), plan)
+ // Offset is equal to zero, so Offset's stats is equal to its child's stats.
+ checkStats(offset, plan.stats.copy(attributeStats = AttributeMap(Nil)))
+ }
+
test("limit estimation: limit < child's rowCount") {
val localLimit = LocalLimit(Literal(2), plan)
val globalLimit = GlobalLimit(Literal(2), plan)
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
index 785566f1ff7f4..96204aa2f071b 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
@@ -1952,9 +1952,16 @@ class Dataset[T] private[sql](
Limit(Literal(n), logicalPlan)
}
- def limitRange(start: Int, end: Int): Dataset[T] = withTypedPlan {
- Limit(Literal(end - start), Offset(Literal(start), logicalPlan))
+ /**
+ * Returns a new Dataset by skipping the first `n` rows.
+ *
+ * @group typedrel
+ * @since 3.4.0
+ */
+ def offset(n: Int): Dataset[T] = withTypedPlan {
+ Offset(Literal(n), logicalPlan)
}
+
/**
* Returns a new Dataset containing union of rows in this Dataset and another Dataset.
*
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/util/V2ExpressionBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/util/V2ExpressionBuilder.scala
index 120b204413561..1a9b9202cbe97 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/util/V2ExpressionBuilder.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/util/V2ExpressionBuilder.scala
@@ -17,19 +17,15 @@
package org.apache.spark.sql.catalyst.util
-import org.apache.spark.sql.catalyst.expressions.{Abs, Add, And, BinaryComparison, BinaryOperator, BitwiseAnd, BitwiseNot, BitwiseOr, BitwiseXor, CaseWhen, Cast, Ceil, Coalesce, Contains, Divide, EndsWith, EqualTo, Exp, Expression, Floor, In, InSet, IsNotNull, IsNull, Literal, Log, Lower, Multiply, Not, Or, Overlay, Pow, Predicate, Remainder, Sqrt, StartsWith, StringPredicate, StringTranslate, StringTrim, StringTrimLeft, StringTrimRight, Substring, Subtract, UnaryMinus, Upper, WidthBucket}
+import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.connector.expressions.{Cast => V2Cast, Expression => V2Expression, FieldReference, GeneralScalarExpression, LiteralValue}
import org.apache.spark.sql.connector.expressions.filter.{AlwaysFalse, AlwaysTrue, And => V2And, Not => V2Not, Or => V2Or, Predicate => V2Predicate}
-import org.apache.spark.sql.execution.datasources.PushableColumn
import org.apache.spark.sql.types.BooleanType
/**
* The builder to generate V2 expressions from catalyst expressions.
*/
-class V2ExpressionBuilder(
- e: Expression, nestedPredicatePushdownEnabled: Boolean = false, isPredicate: Boolean = false) {
-
- val pushableColumn = PushableColumn(nestedPredicatePushdownEnabled)
+class V2ExpressionBuilder(e: Expression, isPredicate: Boolean = false) {
def build(): Option[V2Expression] = generateExpression(e, isPredicate)
@@ -49,12 +45,8 @@ class V2ExpressionBuilder(
case Literal(true, BooleanType) => Some(new AlwaysTrue())
case Literal(false, BooleanType) => Some(new AlwaysFalse())
case Literal(value, dataType) => Some(LiteralValue(value, dataType))
- case col @ pushableColumn(name) =>
- val ref = if (nestedPredicatePushdownEnabled) {
- FieldReference(name)
- } else {
- FieldReference.column(name)
- }
+ case col @ ColumnOrField(nameParts) =>
+ val ref = FieldReference(nameParts)
if (isPredicate && col.dataType.isInstanceOf[BooleanType]) {
Some(new V2Predicate("=", Array(ref, LiteralValue(true, BooleanType))))
} else {
@@ -107,6 +99,39 @@ class V2ExpressionBuilder(
} else {
None
}
+ case Greatest(children) =>
+ val childrenExpressions = children.flatMap(generateExpression(_))
+ if (children.length == childrenExpressions.length) {
+ Some(new GeneralScalarExpression("GREATEST", childrenExpressions.toArray[V2Expression]))
+ } else {
+ None
+ }
+ case Least(children) =>
+ val childrenExpressions = children.flatMap(generateExpression(_))
+ if (children.length == childrenExpressions.length) {
+ Some(new GeneralScalarExpression("LEAST", childrenExpressions.toArray[V2Expression]))
+ } else {
+ None
+ }
+ case Rand(child, hideSeed) =>
+ if (hideSeed) {
+ Some(new GeneralScalarExpression("RAND", Array.empty[V2Expression]))
+ } else {
+ generateExpression(child)
+ .map(v => new GeneralScalarExpression("RAND", Array[V2Expression](v)))
+ }
+ case log: Logarithm =>
+ val l = generateExpression(log.left)
+ val r = generateExpression(log.right)
+ if (l.isDefined && r.isDefined) {
+ Some(new GeneralScalarExpression("LOG", Array[V2Expression](l.get, r.get)))
+ } else {
+ None
+ }
+ case Log10(child) => generateExpression(child)
+ .map(v => new GeneralScalarExpression("LOG10", Array[V2Expression](v)))
+ case Log2(child) => generateExpression(child)
+ .map(v => new GeneralScalarExpression("LOG2", Array[V2Expression](v)))
case Log(child) => generateExpression(child)
.map(v => new GeneralScalarExpression("LN", Array[V2Expression](v)))
case Exp(child) => generateExpression(child)
@@ -125,6 +150,56 @@ class V2ExpressionBuilder(
.map(v => new GeneralScalarExpression("FLOOR", Array[V2Expression](v)))
case Ceil(child) => generateExpression(child)
.map(v => new GeneralScalarExpression("CEIL", Array[V2Expression](v)))
+ case round: Round =>
+ val l = generateExpression(round.left)
+ val r = generateExpression(round.right)
+ if (l.isDefined && r.isDefined) {
+ Some(new GeneralScalarExpression("ROUND", Array[V2Expression](l.get, r.get)))
+ } else {
+ None
+ }
+ case Sin(child) => generateExpression(child)
+ .map(v => new GeneralScalarExpression("SIN", Array[V2Expression](v)))
+ case Sinh(child) => generateExpression(child)
+ .map(v => new GeneralScalarExpression("SINH", Array[V2Expression](v)))
+ case Cos(child) => generateExpression(child)
+ .map(v => new GeneralScalarExpression("COS", Array[V2Expression](v)))
+ case Cosh(child) => generateExpression(child)
+ .map(v => new GeneralScalarExpression("COSH", Array[V2Expression](v)))
+ case Tan(child) => generateExpression(child)
+ .map(v => new GeneralScalarExpression("TAN", Array[V2Expression](v)))
+ case Tanh(child) => generateExpression(child)
+ .map(v => new GeneralScalarExpression("TANH", Array[V2Expression](v)))
+ case Cot(child) => generateExpression(child)
+ .map(v => new GeneralScalarExpression("COT", Array[V2Expression](v)))
+ case Asin(child) => generateExpression(child)
+ .map(v => new GeneralScalarExpression("ASIN", Array[V2Expression](v)))
+ case Asinh(child) => generateExpression(child)
+ .map(v => new GeneralScalarExpression("ASINH", Array[V2Expression](v)))
+ case Acos(child) => generateExpression(child)
+ .map(v => new GeneralScalarExpression("ACOS", Array[V2Expression](v)))
+ case Acosh(child) => generateExpression(child)
+ .map(v => new GeneralScalarExpression("ACOSH", Array[V2Expression](v)))
+ case Atan(child) => generateExpression(child)
+ .map(v => new GeneralScalarExpression("ATAN", Array[V2Expression](v)))
+ case Atanh(child) => generateExpression(child)
+ .map(v => new GeneralScalarExpression("ATANH", Array[V2Expression](v)))
+ case atan2: Atan2 =>
+ val l = generateExpression(atan2.left)
+ val r = generateExpression(atan2.right)
+ if (l.isDefined && r.isDefined) {
+ Some(new GeneralScalarExpression("ATAN2", Array[V2Expression](l.get, r.get)))
+ } else {
+ None
+ }
+ case Cbrt(child) => generateExpression(child)
+ .map(v => new GeneralScalarExpression("CBRT", Array[V2Expression](v)))
+ case ToDegrees(child) => generateExpression(child)
+ .map(v => new GeneralScalarExpression("DEGREES", Array[V2Expression](v)))
+ case ToRadians(child) => generateExpression(child)
+ .map(v => new GeneralScalarExpression("RADIANS", Array[V2Expression](v)))
+ case Signum(child) => generateExpression(child)
+ .map(v => new GeneralScalarExpression("SIGN", Array[V2Expression](v)))
case wb: WidthBucket =>
val childrenExpressions = wb.children.flatMap(generateExpression(_))
if (childrenExpressions.length == wb.children.length) {
@@ -203,6 +278,13 @@ class V2ExpressionBuilder(
} else {
None
}
+ case iff: If =>
+ val childrenExpressions = iff.children.flatMap(generateExpression(_))
+ if (iff.children.length == childrenExpressions.length) {
+ Some(new GeneralScalarExpression("CASE_WHEN", childrenExpressions.toArray[V2Expression]))
+ } else {
+ None
+ }
case substring: Substring =>
val children = if (substring.len == Literal(Integer.MAX_VALUE)) {
Seq(substring.str, substring.pos)
@@ -266,3 +348,12 @@ class V2ExpressionBuilder(
case _ => None
}
}
+
+object ColumnOrField {
+ def unapply(e: Expression): Option[Seq[String]] = e match {
+ case a: Attribute => Some(Seq(a.name))
+ case s: GetStructField =>
+ unapply(s.child).map(_ :+ s.childSchema(s.ordinal).name)
+ case _ => None
+ }
+}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala
index 86edb5acbe292..1181ad05bb413 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala
@@ -148,9 +148,11 @@ case class RowDataSourceScanExec(
s"ORDER BY ${seqToString(pushedDownOperators.sortValues.map(_.describe()))}" +
s" LIMIT ${pushedDownOperators.limit.get}"
Some("PushedTopN" -> pushedTopN)
- } else {
- pushedDownOperators.limit.map(value => "PushedLimit" -> s"LIMIT $value")
- }
+ } else {
+ pushedDownOperators.limit.map(value => "PushedLimit" -> s"LIMIT $value")
+ }
+
+ val offsetInfo = pushedDownOperators.offset.map(value => "PushedOffset" -> s"OFFSET $value")
val pushedFilters = if (pushedDownOperators.pushedPredicates.nonEmpty) {
seqToString(pushedDownOperators.pushedPredicates.map(_.describe()))
@@ -164,6 +166,7 @@ case class RowDataSourceScanExec(
Map("PushedAggregates" -> seqToString(v.aggregateExpressions.map(_.describe())),
"PushedGroupByExpressions" -> seqToString(v.groupByExpressions.map(_.describe())))} ++
topNOrLimitInfo ++
+ offsetInfo ++
pushedDownOperators.sample.map(v => "PushedSample" ->
s"SAMPLE (${(v.upperBound - v.lowerBound) * 100}) ${v.withReplacement} SEED(${v.seed})"
)
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 8b5ee84fb6e98..422f1f041a58b 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
@@ -80,40 +80,56 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] {
*/
object SpecialLimits extends Strategy {
override def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match {
- case ReturnAnswer(rootPlan) => rootPlan match {
- case Limit(IntegerLiteral(limit), Sort(order, true, child))
- if limit < conf.topKSortFallbackThreshold =>
- TakeOrderedAndProjectExec(limit, 0, order, child.output, planLater(child)) :: Nil
- case Limit(IntegerLiteral(limit), Project(projectList, Sort(order, true, child)))
- if limit < conf.topKSortFallbackThreshold =>
- TakeOrderedAndProjectExec(limit, 0, order, projectList, planLater(child)) :: Nil
+ // Call `planTakeOrdered` first which matches a larger plan.
+ case ReturnAnswer(rootPlan) => planTakeOrdered(rootPlan).getOrElse(rootPlan match {
+ // We should match the combination of limit and offset first, to get the optimal physical
+ // plan, instead of planning limit and offset separately.
+ case LimitAndOffset(limit, offset, child) =>
+ CollectLimitExec(limit = limit, child = planLater(child), offset = offset)
+ case OffsetAndLimit(offset, limit, child) =>
+ // 'Offset a' then 'Limit b' is the same as 'Limit a + b' then 'Offset a'.
+ CollectLimitExec(limit = offset + limit, child = planLater(child), offset = offset)
case Limit(IntegerLiteral(limit), child) =>
- CollectLimitExec(limit, 0, planLater(child)) :: Nil
- case GlobalLimitAndOffset(
- IntegerLiteral(limit),
- IntegerLiteral(offset),
- Sort(order, true, child))
+ CollectLimitExec(limit = limit, child = planLater(child))
+ case logical.Offset(IntegerLiteral(offset), child) =>
+ CollectLimitExec(child = planLater(child), offset = offset)
+ case Tail(IntegerLiteral(limit), child) =>
+ CollectTailExec(limit, planLater(child))
+ case other => planLater(other)
+ }) :: Nil
+
+ case other => planTakeOrdered(other).toSeq
+ }
+
+ private def planTakeOrdered(plan: LogicalPlan): Option[SparkPlan] = plan match {
+ // We should match the combination of limit and offset first, to get the optimal physical
+ // plan, instead of planning limit and offset separately.
+ case LimitAndOffset(limit, offset, Sort(order, true, child))
if limit < conf.topKSortFallbackThreshold =>
- TakeOrderedAndProjectExec(limit, offset, order, child.output, planLater(child)) :: Nil
- case GlobalLimitAndOffset(
- IntegerLiteral(limit),
- IntegerLiteral(offset),
- Project(projectList, Sort(order, true, child)))
+ Some(TakeOrderedAndProjectExec(
+ limit, order, child.output, planLater(child), offset))
+ case LimitAndOffset(limit, offset, Project(projectList, Sort(order, true, child)))
if limit < conf.topKSortFallbackThreshold =>
- TakeOrderedAndProjectExec(limit, offset, order, projectList, planLater(child)) :: Nil
- case GlobalLimitAndOffset(IntegerLiteral(limit), IntegerLiteral(offset), child) =>
- CollectLimitExec(limit, offset, planLater(child)) :: Nil
- case Tail(IntegerLiteral(limit), child) =>
- CollectTailExec(limit, planLater(child)) :: Nil
- case other => planLater(other) :: Nil
- }
+ Some(TakeOrderedAndProjectExec(
+ limit, order, projectList, planLater(child), offset))
+ // 'Offset a' then 'Limit b' is the same as 'Limit a + b' then 'Offset a'.
+ case OffsetAndLimit(offset, limit, Sort(order, true, child))
+ if offset + limit < conf.topKSortFallbackThreshold =>
+ Some(TakeOrderedAndProjectExec(
+ offset + limit, order, child.output, planLater(child), offset))
+ case OffsetAndLimit(offset, limit, Project(projectList, Sort(order, true, child)))
+ if offset + limit < conf.topKSortFallbackThreshold =>
+ Some(TakeOrderedAndProjectExec(
+ offset + limit, order, projectList, planLater(child), offset))
case Limit(IntegerLiteral(limit), Sort(order, true, child))
if limit < conf.topKSortFallbackThreshold =>
- TakeOrderedAndProjectExec(limit, 0, order, child.output, planLater(child)) :: Nil
+ Some(TakeOrderedAndProjectExec(
+ limit, order, child.output, planLater(child)))
case Limit(IntegerLiteral(limit), Project(projectList, Sort(order, true, child)))
if limit < conf.topKSortFallbackThreshold =>
- TakeOrderedAndProjectExec(limit, 0, order, projectList, planLater(child)) :: Nil
- case _ => Nil
+ Some(TakeOrderedAndProjectExec(
+ limit, order, projectList, planLater(child)))
+ case _ => None
}
}
@@ -771,12 +787,19 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] {
case logical.LocalRelation(output, data, _) =>
LocalTableScanExec(output, data) :: Nil
case CommandResult(output, _, plan, data) => CommandResultExec(output, plan, data) :: Nil
+ // We should match the combination of limit and offset first, to get the optimal physical
+ // plan, instead of planning limit and offset separately.
+ case LimitAndOffset(limit, offset, child) =>
+ GlobalLimitExec(limit, planLater(child), offset) :: Nil
+ case OffsetAndLimit(offset, limit, child) =>
+ // 'Offset a' then 'Limit b' is the same as 'Limit a + b' then 'Offset a'.
+ GlobalLimitExec(limit = offset + limit, child = planLater(child), offset = offset) :: Nil
case logical.LocalLimit(IntegerLiteral(limit), child) =>
execution.LocalLimitExec(limit, planLater(child)) :: Nil
case logical.GlobalLimit(IntegerLiteral(limit), child) =>
execution.GlobalLimitExec(limit, planLater(child)) :: Nil
- case logical.GlobalLimitAndOffset(IntegerLiteral(limit), IntegerLiteral(offset), child) =>
- execution.GlobalLimitAndOffsetExec(limit, offset, planLater(child)) :: Nil
+ case logical.Offset(IntegerLiteral(offset), child) =>
+ GlobalLimitExec(child = planLater(child), offset = offset) :: Nil
case union: logical.Union =>
execution.UnionExec(union.children.map(planLater)) :: Nil
case g @ logical.Generate(generator, _, outer, _, _, child) =>
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala
index 7209a61d39a31..8d8e2c26e279e 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala
@@ -337,7 +337,7 @@ object DataSourceStrategy
l.output.toStructType,
Set.empty,
Set.empty,
- PushedDownOperators(None, None, None, Seq.empty, Seq.empty),
+ PushedDownOperators(None, None, None, None, Seq.empty, Seq.empty),
toCatalystRDD(l, baseRelation.buildScan()),
baseRelation,
None) :: Nil
@@ -411,7 +411,7 @@ object DataSourceStrategy
requestedColumns.toStructType,
pushedFilters.toSet,
handledFilters,
- PushedDownOperators(None, None, None, Seq.empty, Seq.empty),
+ PushedDownOperators(None, None, None, None, Seq.empty, Seq.empty),
scanBuilder(requestedColumns, candidatePredicates, pushedFilters),
relation.relation,
relation.catalogTable.map(_.identifier))
@@ -434,7 +434,7 @@ object DataSourceStrategy
requestedColumns.toStructType,
pushedFilters.toSet,
handledFilters,
- PushedDownOperators(None, None, None, Seq.empty, Seq.empty),
+ PushedDownOperators(None, None, None, None, Seq.empty, Seq.empty),
scanBuilder(requestedColumns, candidatePredicates, pushedFilters),
relation.relation,
relation.catalogTable.map(_.identifier))
@@ -770,8 +770,8 @@ object DataSourceStrategy
}
protected[sql] def translateSortOrders(sortOrders: Seq[SortOrder]): Seq[V2SortOrder] = {
- def translateOortOrder(sortOrder: SortOrder): Option[V2SortOrder] = sortOrder match {
- case SortOrder(PushableColumnWithoutNestedColumn(name), directionV1, nullOrderingV1, _) =>
+ def translateSortOrder(sortOrder: SortOrder): Option[V2SortOrder] = sortOrder match {
+ case SortOrder(PushableExpression(expr), directionV1, nullOrderingV1, _) =>
val directionV2 = directionV1 match {
case Ascending => SortDirection.ASCENDING
case Descending => SortDirection.DESCENDING
@@ -780,11 +780,11 @@ object DataSourceStrategy
case NullsFirst => NullOrdering.NULLS_FIRST
case NullsLast => NullOrdering.NULLS_LAST
}
- Some(SortValue(FieldReference(name), directionV2, nullOrderingV2))
+ Some(SortValue(expr, directionV2, nullOrderingV2))
case _ => None
}
- sortOrders.flatMap(translateOortOrder)
+ sortOrders.flatMap(translateSortOrder)
}
/**
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCOptions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCOptions.scala
index 8e047d7f7c7d5..5cf38eb803216 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCOptions.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCOptions.scala
@@ -195,6 +195,10 @@ class JDBCOptions(
// This only applies to Data Source V2 JDBC
val pushDownLimit = parameters.getOrElse(JDBC_PUSHDOWN_LIMIT, "false").toBoolean
+ // An option to allow/disallow pushing down OFFSET into V2 JDBC data source
+ // This only applies to Data Source V2 JDBC
+ val pushDownOffset = parameters.getOrElse(JDBC_PUSHDOWN_OFFSET, "false").toBoolean
+
// An option to allow/disallow pushing down TABLESAMPLE into JDBC data source
// This only applies to Data Source V2 JDBC
val pushDownTableSample = parameters.getOrElse(JDBC_PUSHDOWN_TABLESAMPLE, "false").toBoolean
@@ -272,6 +276,7 @@ object JDBCOptions {
val JDBC_PUSHDOWN_PREDICATE = newOption("pushDownPredicate")
val JDBC_PUSHDOWN_AGGREGATE = newOption("pushDownAggregate")
val JDBC_PUSHDOWN_LIMIT = newOption("pushDownLimit")
+ val JDBC_PUSHDOWN_OFFSET = newOption("pushDownOffset")
val JDBC_PUSHDOWN_TABLESAMPLE = newOption("pushDownTableSample")
val JDBC_KEYTAB = newOption("keytab")
val JDBC_PRINCIPAL = newOption("principal")
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRDD.scala
index b30b460ac67db..8aba40e7548af 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRDD.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRDD.scala
@@ -25,7 +25,6 @@ import org.apache.spark.{InterruptibleIterator, Partition, SparkContext, TaskCon
import org.apache.spark.internal.Logging
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.InternalRow
-import org.apache.spark.sql.connector.expressions.SortOrder
import org.apache.spark.sql.connector.expressions.filter.Predicate
import org.apache.spark.sql.execution.datasources.v2.TableSampleInfo
import org.apache.spark.sql.jdbc.{JdbcDialect, JdbcDialects}
@@ -123,7 +122,8 @@ object JDBCRDD extends Logging {
groupByColumns: Option[Array[String]] = None,
sample: Option[TableSampleInfo] = None,
limit: Int = 0,
- sortOrders: Array[SortOrder] = Array.empty[SortOrder]): RDD[InternalRow] = {
+ sortOrders: Array[String] = Array.empty[String],
+ offset: Int = 0): RDD[InternalRow] = {
val url = options.url
val dialect = JdbcDialects.get(url)
val quotedColumns = if (groupByColumns.isEmpty) {
@@ -144,7 +144,8 @@ object JDBCRDD extends Logging {
groupByColumns,
sample,
limit,
- sortOrders)
+ sortOrders,
+ offset)
}
// scalastyle:on argcount
}
@@ -166,7 +167,8 @@ private[jdbc] class JDBCRDD(
groupByColumns: Option[Array[String]],
sample: Option[TableSampleInfo],
limit: Int,
- sortOrders: Array[SortOrder])
+ sortOrders: Array[String],
+ offset: Int)
extends RDD[InternalRow](sc, Nil) {
/**
@@ -216,7 +218,7 @@ private[jdbc] class JDBCRDD(
private def getOrderByClause: String = {
if (sortOrders.nonEmpty) {
- s" ORDER BY ${sortOrders.map(_.describe()).mkString(", ")}"
+ s" ORDER BY ${sortOrders.mkString(", ")}"
} else {
""
}
@@ -304,9 +306,10 @@ private[jdbc] class JDBCRDD(
}
val myLimitClause: String = dialect.getLimitClause(limit)
+ val myOffsetClause: String = dialect.getOffsetClause(offset)
val sqlText = s"SELECT $columnList FROM ${options.tableOrQuery} $myTableSampleClause" +
- s" $myWhereClause $getGroupByClause $getOrderByClause $myLimitClause"
+ s" $myWhereClause $getGroupByClause $getOrderByClause $myLimitClause $myOffsetClause"
stmt = conn.prepareStatement(sqlText,
ResultSet.TYPE_FORWARD_ONLY, ResultSet.CONCUR_READ_ONLY)
stmt.setFetchSize(options.fetchSize)
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala
index 0f1a1b6dc667b..fc439ac5ea1f3 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala
@@ -27,7 +27,6 @@ import org.apache.spark.sql.{DataFrame, Row, SaveMode, SparkSession, SQLContext}
import org.apache.spark.sql.catalyst.analysis._
import org.apache.spark.sql.catalyst.util.{DateFormatter, DateTimeUtils, TimestampFormatter}
import org.apache.spark.sql.catalyst.util.DateTimeUtils.{getZoneId, stringToDate, stringToTimestamp}
-import org.apache.spark.sql.connector.expressions.SortOrder
import org.apache.spark.sql.connector.expressions.filter.Predicate
import org.apache.spark.sql.errors.QueryCompilationErrors
import org.apache.spark.sql.execution.datasources.v2.TableSampleInfo
@@ -305,7 +304,8 @@ private[sql] case class JDBCRelation(
groupByColumns: Option[Array[String]],
tableSample: Option[TableSampleInfo],
limit: Int,
- sortOrders: Array[SortOrder]): RDD[Row] = {
+ sortOrders: Array[String],
+ offset: Int): RDD[Row] = {
// Rely on a type erasure hack to pass RDD[InternalRow] back as RDD[Row]
JDBCRDD.scanTable(
sparkSession.sparkContext,
@@ -318,7 +318,8 @@ private[sql] case class JDBCRelation(
groupByColumns,
tableSample,
limit,
- sortOrders).asInstanceOf[RDD[Row]]
+ sortOrders,
+ offset).asInstanceOf[RDD[Row]]
}
override def insert(data: DataFrame, overwrite: Boolean): Unit = {
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala
index f267a03cbe218..7430cb43db985 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala
@@ -434,12 +434,9 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat
private[sql] object DataSourceV2Strategy {
- private def translateLeafNodeFilterV2(
- predicate: Expression,
- supportNestedPredicatePushdown: Boolean): Option[Predicate] = {
- val pushablePredicate = PushablePredicate(supportNestedPredicatePushdown)
+ private def translateLeafNodeFilterV2(predicate: Expression): Option[Predicate] = {
predicate match {
- case pushablePredicate(expr) => Some(expr)
+ case PushablePredicate(expr) => Some(expr)
case _ => None
}
}
@@ -449,10 +446,8 @@ private[sql] object DataSourceV2Strategy {
*
* @return a `Some[Filter]` if the input [[Expression]] is convertible, otherwise a `None`.
*/
- protected[sql] def translateFilterV2(
- predicate: Expression,
- supportNestedPredicatePushdown: Boolean): Option[Predicate] = {
- translateFilterV2WithMapping(predicate, None, supportNestedPredicatePushdown)
+ protected[sql] def translateFilterV2(predicate: Expression): Option[Predicate] = {
+ translateFilterV2WithMapping(predicate, None)
}
/**
@@ -466,8 +461,7 @@ private[sql] object DataSourceV2Strategy {
*/
protected[sql] def translateFilterV2WithMapping(
predicate: Expression,
- translatedFilterToExpr: Option[mutable.HashMap[Predicate, Expression]],
- nestedPredicatePushdownEnabled: Boolean)
+ translatedFilterToExpr: Option[mutable.HashMap[Predicate, Expression]])
: Option[Predicate] = {
predicate match {
case And(left, right) =>
@@ -481,26 +475,21 @@ private[sql] object DataSourceV2Strategy {
// Pushing one leg of AND down is only safe to do at the top level.
// You can see ParquetFilters' createFilter for more details.
for {
- leftFilter <- translateFilterV2WithMapping(
- left, translatedFilterToExpr, nestedPredicatePushdownEnabled)
- rightFilter <- translateFilterV2WithMapping(
- right, translatedFilterToExpr, nestedPredicatePushdownEnabled)
+ leftFilter <- translateFilterV2WithMapping(left, translatedFilterToExpr)
+ rightFilter <- translateFilterV2WithMapping(right, translatedFilterToExpr)
} yield new V2And(leftFilter, rightFilter)
case Or(left, right) =>
for {
- leftFilter <- translateFilterV2WithMapping(
- left, translatedFilterToExpr, nestedPredicatePushdownEnabled)
- rightFilter <- translateFilterV2WithMapping(
- right, translatedFilterToExpr, nestedPredicatePushdownEnabled)
+ leftFilter <- translateFilterV2WithMapping(left, translatedFilterToExpr)
+ rightFilter <- translateFilterV2WithMapping(right, translatedFilterToExpr)
} yield new V2Or(leftFilter, rightFilter)
case Not(child) =>
- translateFilterV2WithMapping(child, translatedFilterToExpr, nestedPredicatePushdownEnabled)
- .map(new V2Not(_))
+ translateFilterV2WithMapping(child, translatedFilterToExpr).map(new V2Not(_))
case other =>
- val filter = translateLeafNodeFilterV2(other, nestedPredicatePushdownEnabled)
+ val filter = translateLeafNodeFilterV2(other)
if (filter.isDefined && translatedFilterToExpr.isDefined) {
translatedFilterToExpr.get(filter.get) = predicate
}
@@ -532,10 +521,9 @@ private[sql] object DataSourceV2Strategy {
/**
* Get the expression of DS V2 to represent catalyst predicate that can be pushed down.
*/
-case class PushablePredicate(nestedPredicatePushdownEnabled: Boolean) {
-
+object PushablePredicate {
def unapply(e: Expression): Option[Predicate] =
- new V2ExpressionBuilder(e, nestedPredicatePushdownEnabled, true).build().map { v =>
+ new V2ExpressionBuilder(e, true).build().map { v =>
assert(v.isInstanceOf[Predicate])
v.asInstanceOf[Predicate]
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PushDownUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PushDownUtils.scala
index 34bc384b318d9..66d26446f40a6 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PushDownUtils.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PushDownUtils.scala
@@ -23,7 +23,7 @@ import org.apache.spark.sql.catalyst.expressions.{AttributeReference, AttributeS
import org.apache.spark.sql.catalyst.util.CharVarcharUtils
import org.apache.spark.sql.connector.expressions.SortOrder
import org.apache.spark.sql.connector.expressions.filter.Predicate
-import org.apache.spark.sql.connector.read.{Scan, ScanBuilder, SupportsPushDownFilters, SupportsPushDownLimit, SupportsPushDownRequiredColumns, SupportsPushDownTableSample, SupportsPushDownTopN, SupportsPushDownV2Filters}
+import org.apache.spark.sql.connector.read.{Scan, ScanBuilder, SupportsPushDownFilters, SupportsPushDownLimit, SupportsPushDownOffset, SupportsPushDownRequiredColumns, SupportsPushDownTableSample, SupportsPushDownTopN, SupportsPushDownV2Filters}
import org.apache.spark.sql.execution.datasources.DataSourceStrategy
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.sources
@@ -80,7 +80,7 @@ object PushDownUtils extends PredicateHelper {
for (filterExpr <- filters) {
val translated =
DataSourceV2Strategy.translateFilterV2WithMapping(
- filterExpr, Some(translatedFilterToExpr), nestedPredicatePushdownEnabled = true)
+ filterExpr, Some(translatedFilterToExpr))
if (translated.isEmpty) {
untranslatableExprs += filterExpr
} else {
@@ -131,6 +131,19 @@ object PushDownUtils extends PredicateHelper {
}
}
+ /**
+ * Pushes down OFFSET to the data source Scan.
+ *
+ * @return the Boolean value represents whether to push down.
+ */
+ def pushOffset(scanBuilder: ScanBuilder, offset: Int): Boolean = {
+ scanBuilder match {
+ case s: SupportsPushDownOffset =>
+ s.pushOffset(offset)
+ case _ => false
+ }
+ }
+
/**
* Pushes down top N to the data source Scan.
*
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PushedDownOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PushedDownOperators.scala
index a95b4593fc397..49044c6e24db6 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PushedDownOperators.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PushedDownOperators.scala
@@ -28,6 +28,7 @@ case class PushedDownOperators(
aggregation: Option[Aggregation],
sample: Option[TableSampleInfo],
limit: Option[Int],
+ offset: Option[Int],
sortValues: Seq[SortOrder],
pushedPredicates: Seq[Predicate]) {
assert((limit.isEmpty && sortValues.isEmpty) || limit.isDefined)
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ScanRelationPushDown.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ScanRelationPushDown.scala
index 3a3896d9c9fe8..f69cf937b099b 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ScanRelationPushDown.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ScanRelationPushDown.scala
@@ -19,12 +19,11 @@ package org.apache.spark.sql.execution.datasources.v2
import scala.collection.mutable
-import org.apache.spark.sql.catalyst.expressions.{Alias, AliasHelper, And, Attribute, AttributeReference, Cast, Expression, IntegerLiteral, NamedExpression, PredicateHelper, ProjectionOverSchema, SortOrder, SubqueryExpression}
-import org.apache.spark.sql.catalyst.expressions.aggregate
+import org.apache.spark.sql.catalyst.expressions.{aggregate, Alias, AliasHelper, And, Attribute, AttributeReference, Cast, Expression, IntegerLiteral, Literal, NamedExpression, PredicateHelper, ProjectionOverSchema, SortOrder, SubqueryExpression}
import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
import org.apache.spark.sql.catalyst.optimizer.CollapseProject
import org.apache.spark.sql.catalyst.planning.ScanOperation
-import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, LeafNode, Limit, LocalLimit, LogicalPlan, Project, Sample, Sort}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, LeafNode, Limit, LimitAndOffset, LocalLimit, LogicalPlan, Offset, OffsetAndLimit, Project, Sample, Sort}
import org.apache.spark.sql.catalyst.rules.Rule
import org.apache.spark.sql.connector.expressions.{SortOrder => V2SortOrder}
import org.apache.spark.sql.connector.expressions.aggregate.{Aggregation, Avg, Count, GeneralAggregateFunc, Sum}
@@ -32,15 +31,24 @@ import org.apache.spark.sql.connector.expressions.filter.Predicate
import org.apache.spark.sql.connector.read.{Scan, ScanBuilder, SupportsPushDownAggregates, SupportsPushDownFilters, V1Scan}
import org.apache.spark.sql.execution.datasources.DataSourceStrategy
import org.apache.spark.sql.sources
-import org.apache.spark.sql.types.{DataType, LongType, StructType}
+import org.apache.spark.sql.types.{DataType, IntegerType, LongType, StructType}
import org.apache.spark.sql.util.SchemaUtils._
object V2ScanRelationPushDown extends Rule[LogicalPlan] with PredicateHelper with AliasHelper {
import DataSourceV2Implicits._
def apply(plan: LogicalPlan): LogicalPlan = {
- applyColumnPruning(
- applyLimit(pushDownAggregates(pushDownFilters(pushDownSample(createScanBuilder(plan))))))
+ val pushdownRules = Seq[LogicalPlan => LogicalPlan] (
+ createScanBuilder,
+ pushDownSample,
+ pushDownFilters,
+ pushDownAggregates,
+ pushDownLimitAndOffset,
+ pruneColumns)
+
+ pushdownRules.foldLeft(plan) { (newPlan, pushDownRule) =>
+ pushDownRule(newPlan)
+ }
}
private def createScanBuilder(plan: LogicalPlan) = plan.transform {
@@ -302,7 +310,7 @@ object V2ScanRelationPushDown extends Rule[LogicalPlan] with PredicateHelper wit
Cast(expression, expectedDataType)
}
- def applyColumnPruning(plan: LogicalPlan): LogicalPlan = plan.transform {
+ def pruneColumns(plan: LogicalPlan): LogicalPlan = plan.transform {
case ScanOperation(project, filters, sHolder: ScanBuilderHolder) =>
// column pruning
val normalizedProjects = DataSourceStrategy
@@ -372,7 +380,9 @@ object V2ScanRelationPushDown extends Rule[LogicalPlan] with PredicateHelper wit
order, project, alwaysInline = true) =>
val aliasMap = getAliasMap(project)
val newOrder = order.map(replaceAlias(_, aliasMap)).asInstanceOf[Seq[SortOrder]]
- val orders = DataSourceStrategy.translateSortOrders(newOrder)
+ val normalizedOrders = DataSourceStrategy.normalizeExprs(
+ newOrder, sHolder.relation.output).asInstanceOf[Seq[SortOrder]]
+ val orders = DataSourceStrategy.translateSortOrders(normalizedOrders)
if (orders.length == order.length) {
val (isPushed, isPartiallyPushed) =
PushDownUtils.pushTopN(sHolder.builder, orders.toArray, limit)
@@ -396,7 +406,60 @@ object V2ScanRelationPushDown extends Rule[LogicalPlan] with PredicateHelper wit
case other => (other, false)
}
- def applyLimit(plan: LogicalPlan): LogicalPlan = plan.transform {
+ private def pushDownOffset(
+ plan: LogicalPlan,
+ offset: Int): Boolean = plan match {
+ case sHolder: ScanBuilderHolder =>
+ val isPushed = PushDownUtils.pushOffset(sHolder.builder, offset)
+ if (isPushed) {
+ sHolder.pushedOffset = Some(offset)
+ }
+ isPushed
+ case Project(projectList, child) if projectList.forall(_.deterministic) =>
+ pushDownOffset(child, offset)
+ case _ => false
+ }
+
+ def pushDownLimitAndOffset(plan: LogicalPlan): LogicalPlan = plan.transform {
+ case offset @ LimitAndOffset(limit, offsetValue, child) =>
+ val (newChild, canRemoveLimit) = pushDownLimit(child, limit)
+ if (canRemoveLimit) {
+ // Try to push down OFFSET only if the LIMIT operator has been pushed and can be removed.
+ val isPushed = pushDownOffset(newChild, offsetValue)
+ if (isPushed) {
+ newChild
+ } else {
+ // Keep the OFFSET operator if we failed to push down OFFSET to the data source.
+ offset.withNewChildren(Seq(newChild))
+ }
+ } else {
+ // Keep the OFFSET operator if we can't remove LIMIT operator.
+ offset
+ }
+ case globalLimit @ OffsetAndLimit(offset, limit, child) =>
+ // For `df.offset(n).limit(m)`, we can push down `limit(m + n)` first.
+ val (newChild, canRemoveLimit) = pushDownLimit(child, limit + offset)
+ if (canRemoveLimit) {
+ // Try to push down OFFSET only if the LIMIT operator has been pushed and can be removed.
+ val isPushed = pushDownOffset(newChild, offset)
+ if (isPushed) {
+ newChild
+ } else {
+ // Still keep the OFFSET operator if we can't push it down.
+ Offset(Literal(offset), newChild)
+ }
+ } else {
+ // For `df.offset(n).limit(m)`, since we can't push down `limit(m + n)`,
+ // try to push down `offset(n)` here.
+ val isPushed = pushDownOffset(child, offset)
+ if (isPushed) {
+ // Keep the LIMIT operator if we can't push it down.
+ Limit(Literal(limit, IntegerType), child)
+ } else {
+ // Keep the origin plan if we can't push OFFSET operator and LIMIT operator.
+ globalLimit
+ }
+ }
case globalLimit @ Limit(IntegerLiteral(limitValue), child) =>
val (newChild, canRemoveLimit) = pushDownLimit(child, limitValue)
if (canRemoveLimit) {
@@ -406,6 +469,13 @@ object V2ScanRelationPushDown extends Rule[LogicalPlan] with PredicateHelper wit
globalLimit.child.asInstanceOf[LocalLimit].withNewChildren(Seq(newChild))
globalLimit.withNewChildren(Seq(newLocalLimit))
}
+ case offset @ Offset(IntegerLiteral(n), child) =>
+ val isPushed = pushDownOffset(child, n)
+ if (isPushed) {
+ child
+ } else {
+ offset
+ }
}
private def getWrappedScan(
@@ -420,7 +490,7 @@ object V2ScanRelationPushDown extends Rule[LogicalPlan] with PredicateHelper wit
case _ => Array.empty[sources.Filter]
}
val pushedDownOperators = PushedDownOperators(aggregation, sHolder.pushedSample,
- sHolder.pushedLimit, sHolder.sortOrders, sHolder.pushedPredicates)
+ sHolder.pushedLimit, sHolder.pushedOffset, sHolder.sortOrders, sHolder.pushedPredicates)
V1ScanWrapper(v1, pushedFilters, pushedDownOperators)
case _ => scan
}
@@ -433,6 +503,8 @@ case class ScanBuilderHolder(
builder: ScanBuilder) extends LeafNode {
var pushedLimit: Option[Int] = None
+ var pushedOffset: Option[Int] = None
+
var sortOrders: Seq[V2SortOrder] = Seq.empty[V2SortOrder]
var pushedSample: Option[TableSampleInfo] = None
@@ -440,7 +512,6 @@ case class ScanBuilderHolder(
var pushedPredicates: Seq[Predicate] = Seq.empty[Predicate]
}
-
// A wrapper for v1 scan to carry the translated filters and the handled ones, along with
// other pushed down operators. This is required by the physical v1 scan node.
case class V1ScanWrapper(
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCScan.scala
index f68f78d51fd96..ea642a3a5e510 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCScan.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCScan.scala
@@ -18,7 +18,6 @@ package org.apache.spark.sql.execution.datasources.v2.jdbc
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.{Row, SQLContext}
-import org.apache.spark.sql.connector.expressions.SortOrder
import org.apache.spark.sql.connector.expressions.filter.Predicate
import org.apache.spark.sql.connector.read.V1Scan
import org.apache.spark.sql.execution.datasources.jdbc.JDBCRelation
@@ -34,7 +33,8 @@ case class JDBCScan(
groupByColumns: Option[Array[String]],
tableSample: Option[TableSampleInfo],
pushedLimit: Int,
- sortOrders: Array[SortOrder]) extends V1Scan {
+ sortOrders: Array[String],
+ pushedOffset: Int) extends V1Scan {
override def readSchema(): StructType = prunedSchema
@@ -50,7 +50,7 @@ case class JDBCScan(
pushedAggregateColumn
}
relation.buildScan(columnList, prunedSchema, pushedPredicates, groupByColumns, tableSample,
- pushedLimit, sortOrders)
+ pushedLimit, sortOrders, pushedOffset)
}
}.asInstanceOf[T]
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCScanBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCScanBuilder.scala
index 8b378d2d87c49..cd274e2248529 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCScanBuilder.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCScanBuilder.scala
@@ -23,7 +23,7 @@ import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.connector.expressions.{FieldReference, SortOrder}
import org.apache.spark.sql.connector.expressions.aggregate.Aggregation
import org.apache.spark.sql.connector.expressions.filter.Predicate
-import org.apache.spark.sql.connector.read.{Scan, ScanBuilder, SupportsPushDownAggregates, SupportsPushDownLimit, SupportsPushDownRequiredColumns, SupportsPushDownTableSample, SupportsPushDownTopN, SupportsPushDownV2Filters}
+import org.apache.spark.sql.connector.read.{Scan, ScanBuilder, SupportsPushDownAggregates, SupportsPushDownLimit, SupportsPushDownOffset, SupportsPushDownRequiredColumns, SupportsPushDownTableSample, SupportsPushDownTopN, SupportsPushDownV2Filters}
import org.apache.spark.sql.execution.datasources.PartitioningUtils
import org.apache.spark.sql.execution.datasources.jdbc.{JDBCOptions, JDBCRDD, JDBCRelation}
import org.apache.spark.sql.execution.datasources.v2.TableSampleInfo
@@ -39,6 +39,7 @@ case class JDBCScanBuilder(
with SupportsPushDownRequiredColumns
with SupportsPushDownAggregates
with SupportsPushDownLimit
+ with SupportsPushDownOffset
with SupportsPushDownTableSample
with SupportsPushDownTopN
with Logging {
@@ -53,7 +54,9 @@ case class JDBCScanBuilder(
private var pushedLimit = 0
- private var sortOrders: Array[SortOrder] = Array.empty[SortOrder]
+ private var pushedOffset = 0
+
+ private var sortOrders: Array[String] = Array.empty[String]
override def pushPredicates(predicates: Array[Predicate]): Array[Predicate] = {
if (jdbcOptions.pushDownPredicate) {
@@ -138,10 +141,29 @@ case class JDBCScanBuilder(
false
}
+ override def pushOffset(offset: Int): Boolean = {
+ if (jdbcOptions.pushDownOffset && !isPartiallyPushed) {
+ // Spark pushes down LIMIT first, then OFFSET. In SQL statements, OFFSET is applied before
+ // LIMIT. Here we need to adjust the LIMIT value to match SQL statements.
+ if (pushedLimit > 0) {
+ pushedLimit = pushedLimit - offset
+ }
+ pushedOffset = offset
+ return true
+ }
+ false
+ }
+
override def pushTopN(orders: Array[SortOrder], limit: Int): Boolean = {
if (jdbcOptions.pushDownLimit) {
+ val dialect = JdbcDialects.get(jdbcOptions.url)
+ val compiledOrders = orders.flatMap { order =>
+ dialect.compileExpression(order.expression())
+ .map(sortKey => s"$sortKey ${order.direction()} ${order.nullOrdering()}")
+ }
+ if (orders.length != compiledOrders.length) return false
pushedLimit = limit
- sortOrders = orders
+ sortOrders = compiledOrders
return true
}
false
@@ -174,6 +196,6 @@ case class JDBCScanBuilder(
// prunedSchema and quote them (will become "MAX(SALARY)", "MIN(BONUS)" and can't
// be used in sql string.
JDBCScan(JDBCRelation(schema, parts, jdbcOptions)(session), finalSchema, pushedPredicate,
- pushedAggregateList, pushedGroupBys, tableSample, pushedLimit, sortOrders)
+ pushedAggregateList, pushedGroupBys, tableSample, pushedLimit, sortOrders, pushedOffset)
}
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala
index f2204c0c1e9c3..dbba19002c564 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala
@@ -37,16 +37,32 @@ trait LimitExec extends UnaryExecNode {
}
/**
- * Take the first `limit` elements and collect them to a single partition.
+ * Take the first `limit` elements, collect them to a single partition and then to drop the
+ * first `offset` elements.
*
- * This operator will be used when a logical `Limit` operation is the final operator in an
- * logical plan, which happens when the user is collecting results back to the driver.
+ * This operator will be used when a logical `Limit` and/or `Offset` operation is the final operator
+ * in an logical plan, which happens when the user is collecting results back to the driver.
*/
-case class CollectLimitExec(limit: Int, offset: Int, child: SparkPlan) extends LimitExec {
+case class CollectLimitExec(limit: Int = -1, child: SparkPlan, offset: Int = 0) extends LimitExec {
+ assert(limit >= 0 || (limit == -1 && offset > 0))
+
override def output: Seq[Attribute] = child.output
override def outputPartitioning: Partitioning = SinglePartition
override def executeCollect(): Array[InternalRow] = {
- child.executeTake(limit + offset).drop(offset)
+ // Because CollectLimitExec collect all the output of child to a single partition, so we need
+ // collect the first `limit` + `offset` elements and then to drop the first `offset` elements.
+ // For example: limit is 1 and offset is 2 and the child output two partition.
+ // The first partition output [1, 2] and the Second partition output [3, 4, 5].
+ // Then [1, 2, 3] will be taken and output [3].
+ if (limit >= 0) {
+ if (offset > 0) {
+ child.executeTake(limit).drop(offset)
+ } else {
+ child.executeTake(limit)
+ }
+ } else {
+ child.executeCollect().drop(offset)
+ }
}
private val serializer: Serializer = new UnsafeRowSerializer(child.output.size)
private lazy val writeMetrics =
@@ -62,7 +78,11 @@ case class CollectLimitExec(limit: Int, offset: Int, child: SparkPlan) extends L
val singlePartitionRDD = if (childRDD.getNumPartitions == 1) {
childRDD
} else {
- val locallyLimited = child.execute().mapPartitionsInternal(_.take(limit + offset))
+ val locallyLimited = if (limit >= 0) {
+ childRDD.mapPartitionsInternal(_.take(limit))
+ } else {
+ childRDD
+ }
new ShuffledRowRDD(
ShuffleExchangeExec.prepareShuffleDependency(
locallyLimited,
@@ -72,10 +92,25 @@ case class CollectLimitExec(limit: Int, offset: Int, child: SparkPlan) extends L
writeMetrics),
readMetrics)
}
- singlePartitionRDD.mapPartitionsInternal(_.drop(offset).take(limit))
+ if (limit >= 0) {
+ if (offset > 0) {
+ singlePartitionRDD.mapPartitionsInternal(_.slice(offset, limit))
+ } else {
+ singlePartitionRDD.mapPartitionsInternal(_.take(limit))
+ }
+ } else {
+ singlePartitionRDD.mapPartitionsInternal(_.drop(offset))
+ }
}
}
+ override def stringArgs: Iterator[Any] = {
+ super.stringArgs.zipWithIndex.filter {
+ case (0, 2) => false
+ case _ => true
+ }.map(_._1)
+ }
+
override protected def withNewChildInternal(newChild: SparkPlan): SparkPlan =
copy(child = newChild)
}
@@ -125,8 +160,8 @@ trait BaseLimitExec extends LimitExec with CodegenSupport {
override def outputOrdering: Seq[SortOrder] = child.outputOrdering
- protected override def doExecute(): RDD[InternalRow] = child.execute().mapPartitions { iter =>
- iter.take(limit)
+ protected override def doExecute(): RDD[InternalRow] = child.execute().mapPartitionsInternal {
+ iter => iter.take(limit)
}
override def inputRDDs(): Seq[RDD[InternalRow]] = {
@@ -137,10 +172,12 @@ trait BaseLimitExec extends LimitExec with CodegenSupport {
// to the parent operator.
override def usedInputs: AttributeSet = AttributeSet.empty
- lazy val countTerm = BaseLimitExec.newLimitCountTerm()
+ protected lazy val countTerm = BaseLimitExec.newLimitCountTerm()
- override lazy val limitNotReachedChecks: Seq[String] = {
+ override lazy val limitNotReachedChecks: Seq[String] = if (limit >= 0) {
s"$countTerm < $limit" +: super.limitNotReachedChecks
+ } else {
+ super.limitNotReachedChecks
}
protected override def doProduce(ctx: CodegenContext): String = {
@@ -174,52 +211,53 @@ case class LocalLimitExec(limit: Int, child: SparkPlan) extends BaseLimitExec {
}
/**
- * Take the first `limit` elements of the child's single output partition.
- */
-case class GlobalLimitExec(limit: Int, child: SparkPlan) extends BaseLimitExec {
-
- override def requiredChildDistribution: List[Distribution] = AllTuples :: Nil
-
- override protected def withNewChildInternal(newChild: SparkPlan): SparkPlan =
- copy(child = newChild)
-}
-
-/**
- * Skip the first `offset` elements then take the first `limit` of the following elements in
- * the child's single output partition.
+ * Take the first `limit` elements and then drop the first `offset` elements in the child's single
+ * output partition.
*/
-case class GlobalLimitAndOffsetExec(
- limit: Int,
- offset: Int,
- child: SparkPlan) extends BaseLimitExec {
+case class GlobalLimitExec(limit: Int = -1, child: SparkPlan, offset: Int = 0)
+ extends BaseLimitExec {
+ assert(limit >= 0 || (limit == -1 && offset > 0))
override def requiredChildDistribution: List[Distribution] = AllTuples :: Nil
- override def outputPartitioning: Partitioning = child.outputPartitioning
-
- override def outputOrdering: Seq[SortOrder] = child.outputOrdering
-
override def doExecute(): RDD[InternalRow] = {
- val rdd = child.execute().mapPartitions { iter => iter.take(limit + offset) }
- rdd.zipWithIndex().filter(_._2 >= offset).map(_._1)
+ if (offset > 0) {
+ if (limit >= 0) {
+ child.execute().mapPartitionsInternal(iter => iter.slice(offset, limit))
+ } else {
+ child.execute().mapPartitionsInternal(iter => iter.drop(offset))
+ }
+ } else {
+ super.doExecute()
+ }
}
- private lazy val skipTerm = BaseLimitExec.newLimitCountTerm()
-
override def doConsume(ctx: CodegenContext, input: Seq[ExprCode], row: ExprCode): String = {
- // The counter name is already obtained by the upstream operators via `limitNotReachedChecks`.
- // Here we have to inline it to not change its name. This is fine as we won't have many limit
- // operators in one query.
- ctx.addMutableState(CodeGenerator.JAVA_INT, skipTerm, forceInline = true, useFreshName = false)
- ctx.addMutableState(CodeGenerator.JAVA_INT, countTerm, forceInline = true, useFreshName = false)
- s"""
- | if ($skipTerm < $offset) {
- | $skipTerm += 1;
- | } else if ($countTerm < $limit) {
- | $countTerm += 1;
- | ${consume(ctx, input)}
- | }
- """.stripMargin
+ if (offset > 0) {
+ val skipTerm = ctx.addMutableState(CodeGenerator.JAVA_INT, "rowsSkipped", forceInline = true)
+ if (limit > 0) {
+ // In codegen, we skip the first `offset` rows, then take the first `limit - offset` rows.
+ val finalLimit = limit - offset
+ s"""
+ | if ($skipTerm < $offset) {
+ | $skipTerm += 1;
+ | } else if ($countTerm < $finalLimit) {
+ | $countTerm += 1;
+ | ${consume(ctx, input)}
+ | }
+ """.stripMargin
+ } else {
+ s"""
+ | if ($skipTerm < $offset) {
+ | $skipTerm += 1;
+ | } else {
+ | ${consume(ctx, input)}
+ | }
+ """.stripMargin
+ }
+ } else {
+ super.doConsume(ctx, input, row)
+ }
}
override protected def withNewChildInternal(newChild: SparkPlan): SparkPlan =
@@ -227,18 +265,18 @@ case class GlobalLimitAndOffsetExec(
}
/**
- * Take the first limit elements as defined by the sortOrder, and do projection if needed.
- * This is logically equivalent to having a Limit operator after a [[SortExec]] operator,
- * or having a [[ProjectExec]] operator between them.
+ * Take the first `limit` elements as defined by the sortOrder, then drop the first `offset`
+ * elements, and do projection if needed. This is logically equivalent to having a Limit and/or
+ * Offset operator after a [[SortExec]] operator, or having a [[ProjectExec]] operator between them.
* This could have been named TopK, but Spark's top operator does the opposite in ordering
* so we name it TakeOrdered to avoid confusion.
*/
case class TakeOrderedAndProjectExec(
limit: Int,
- offset: Int,
sortOrder: Seq[SortOrder],
projectList: Seq[NamedExpression],
- child: SparkPlan) extends UnaryExecNode {
+ child: SparkPlan,
+ offset: Int = 0) extends UnaryExecNode {
override def output: Seq[Attribute] = {
projectList.map(_.toAttribute)
@@ -246,7 +284,8 @@ case class TakeOrderedAndProjectExec(
override def executeCollect(): Array[InternalRow] = {
val ord = new LazilyGeneratedOrdering(sortOrder, child.output)
- val data = child.execute().map(_.copy()).takeOrdered(limit + offset)(ord).drop(offset)
+ val limited = child.execute().mapPartitionsInternal(_.map(_.copy())).takeOrdered(limit)(ord)
+ val data = if (offset > 0) limited.drop(offset) else limited
if (projectList != child.output) {
val proj = UnsafeProjection.create(projectList, child.output)
data.map(r => proj(r).copy())
@@ -272,9 +311,10 @@ case class TakeOrderedAndProjectExec(
val singlePartitionRDD = if (childRDD.getNumPartitions == 1) {
childRDD
} else {
- val localTopK = childRDD.mapPartitions { iter =>
+ val localTopK = childRDD.mapPartitionsInternal { iter =>
Utils.takeOrdered(iter.map(_.copy()), limit)(ord)
}
+
new ShuffledRowRDD(
ShuffleExchangeExec.prepareShuffleDependency(
localTopK,
@@ -284,8 +324,9 @@ case class TakeOrderedAndProjectExec(
writeMetrics),
readMetrics)
}
- singlePartitionRDD.mapPartitions { iter =>
- val topK = Utils.takeOrdered(iter.map(_.copy()), limit)(ord)
+ singlePartitionRDD.mapPartitionsInternal { iter =>
+ val limited = Utils.takeOrdered(iter.map(_.copy()), limit)(ord)
+ val topK = if (offset > 0) limited.drop(offset) else limited
if (projectList != child.output) {
val proj = UnsafeProjection.create(projectList, child.output)
topK.map(r => proj(r))
@@ -307,6 +348,13 @@ case class TakeOrderedAndProjectExec(
s"TakeOrderedAndProject(limit=$limit, orderBy=$orderByString, output=$outputString)"
}
+ override def stringArgs: Iterator[Any] = {
+ super.stringArgs.zipWithIndex.filter {
+ case (0, 4) => false
+ case _ => true
+ }.map(_._1)
+ }
+
override protected def withNewChildInternal(newChild: SparkPlan): SparkPlan =
copy(child = newChild)
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/H2Dialect.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/H2Dialect.scala
index dc834f1d432b1..db24267d4f3fe 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/H2Dialect.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/H2Dialect.scala
@@ -35,8 +35,10 @@ private[sql] object H2Dialect extends JdbcDialect {
url.toLowerCase(Locale.ROOT).startsWith("jdbc:h2")
private val supportedFunctions =
- Set("ABS", "COALESCE", "LN", "EXP", "POWER", "SQRT", "FLOOR", "CEIL",
- "SUBSTRING", "UPPER", "LOWER", "TRANSLATE", "TRIM")
+ Set("ABS", "COALESCE", "GREATEST", "LEAST", "RAND", "LOG", "LOG10", "LN", "EXP",
+ "POWER", "SQRT", "FLOOR", "CEIL", "ROUND", "SIN", "SINH", "COS", "COSH", "TAN",
+ "TANH", "COT", "ASIN", "ACOS", "ATAN", "ATAN2", "DEGREES", "RADIANS", "SIGN",
+ "PI", "SUBSTRING", "UPPER", "LOWER", "TRANSLATE", "TRIM")
override def isSupportedFunction(funcName: String): Boolean =
supportedFunctions.contains(funcName)
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcDialects.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcDialects.scala
index 8115ec292f8d4..d42d4e8fc0bac 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcDialects.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcDialects.scala
@@ -549,6 +549,13 @@ abstract class JdbcDialect extends Serializable with Logging{
if (limit > 0 ) s"LIMIT $limit" else ""
}
+ /**
+ * returns the OFFSET clause for the SELECT statement
+ */
+ def getOffsetClause(offset: Integer): String = {
+ if (offset > 0 ) s"OFFSET $offset" else ""
+ }
+
def supportsTableSample: Boolean = false
def getTableSample(sample: TableSampleInfo): String =
diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/limit.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/limit.sql
index bc0b5d6dddc52..f59575817d6b8 100644
--- a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/limit.sql
+++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/limit.sql
@@ -12,25 +12,24 @@ SELECT '' AS five, unique1, unique2, stringu1
SELECT '' AS two, unique1, unique2, stringu1
FROM onek WHERE unique1 > 60 AND unique1 < 63
ORDER BY unique1 LIMIT 5;
--- [SPARK-28330] ANSI SQL: Top-level in
--- SELECT '' AS three, unique1, unique2, stringu1
--- FROM onek WHERE unique1 > 100
--- ORDER BY unique1 LIMIT 3 OFFSET 20;
--- SELECT '' AS zero, unique1, unique2, stringu1
--- FROM onek WHERE unique1 < 50
--- ORDER BY unique1 DESC LIMIT 8 OFFSET 99;
--- SELECT '' AS eleven, unique1, unique2, stringu1
--- FROM onek WHERE unique1 < 50
--- ORDER BY unique1 DESC LIMIT 20 OFFSET 39;
--- SELECT '' AS ten, unique1, unique2, stringu1
--- FROM onek
--- ORDER BY unique1 OFFSET 990;
+SELECT '' AS three, unique1, unique2, stringu1
+ FROM onek WHERE unique1 > 100
+ ORDER BY unique1 LIMIT 3 OFFSET 20;
+SELECT '' AS zero, unique1, unique2, stringu1
+ FROM onek WHERE unique1 < 50
+ ORDER BY unique1 DESC LIMIT 8 OFFSET 99;
+SELECT '' AS eleven, unique1, unique2, stringu1
+ FROM onek WHERE unique1 < 50
+ ORDER BY unique1 DESC LIMIT 20 OFFSET 39;
+SELECT '' AS ten, unique1, unique2, stringu1
+ FROM onek
+ ORDER BY unique1 OFFSET 990;
-- SELECT '' AS five, unique1, unique2, stringu1
-- FROM onek
-- ORDER BY unique1 OFFSET 990 LIMIT 5;
--- SELECT '' AS five, unique1, unique2, stringu1
--- FROM onek
--- ORDER BY unique1 LIMIT 5 OFFSET 900;
+SELECT '' AS five, unique1, unique2, stringu1
+ FROM onek
+ ORDER BY unique1 LIMIT 5 OFFSET 900;
CREATE OR REPLACE TEMPORARY VIEW INT8_TBL AS SELECT * FROM
(VALUES
@@ -45,8 +44,7 @@ CREATE OR REPLACE TEMPORARY VIEW INT8_TBL AS SELECT * FROM
-- constant, so to ensure executor is exercised, do this:
-- [SPARK-29650] Discard a NULL constant in LIMIT
select * from int8_tbl limit (case when random() < 0.5 then bigint(null) end);
--- [SPARK-28330] ANSI SQL: Top-level in
--- select * from int8_tbl offset (case when random() < 0.5 then bigint(null) end);
+select * from int8_tbl offset (case when random() < 0.5 then bigint(null) end);
-- Test assorted cases involving backwards fetch from a LIMIT plan node
-- [SPARK-20965] Support PREPARE/EXECUTE/DECLARE/FETCH statements
@@ -90,7 +88,7 @@ DROP VIEW INT8_TBL;
-- Stress test for variable LIMIT in conjunction with bounded-heap sorting
--- [SPARK-28330] ANSI SQL: Top-level in
+-- [SPARK-27767] Built-in function: generate_series
-- SELECT
-- (SELECT n
-- FROM (VALUES (1)) AS x,
diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/exists-subquery/exists-orderby-limit.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/exists-subquery/exists-orderby-limit.sql
index 580fc1d4162eb..8e6b49fea8f4e 100644
--- a/sql/core/src/test/resources/sql-tests/inputs/subquery/exists-subquery/exists-orderby-limit.sql
+++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/exists-subquery/exists-orderby-limit.sql
@@ -121,3 +121,81 @@ WHERE NOT EXISTS (SELECT max(dept.dept_id)
WHERE dept.dept_id > 100
GROUP BY state
LIMIT 1);
+
+-- limit and offset in the exists subquery block.
+-- TC.03.01
+SELECT *
+FROM emp
+WHERE EXISTS (SELECT dept.dept_name
+ FROM dept
+ WHERE dept.dept_id > 10
+ LIMIT 1
+ OFFSET 2);
+
+-- limit and offset in the exists subquery block with aggregate.
+-- TC.03.02
+SELECT *
+FROM emp
+WHERE EXISTS (SELECT max(dept.dept_id)
+ FROM dept
+ GROUP BY state
+ LIMIT 1
+ OFFSET 2);
+
+-- limit and offset in the not exists subquery block.
+-- TC.03.03
+SELECT *
+FROM emp
+WHERE NOT EXISTS (SELECT dept.dept_name
+ FROM dept
+ WHERE dept.dept_id > 100
+ LIMIT 1
+ OFFSET 2);
+
+-- limit and offset in the not exists subquery block with aggregates.
+-- TC.03.04
+SELECT *
+FROM emp
+WHERE NOT EXISTS (SELECT max(dept.dept_id)
+ FROM dept
+ WHERE dept.dept_id > 100
+ GROUP BY state
+ LIMIT 1
+ OFFSET 2);
+
+-- offset in the exists subquery block.
+-- TC.04.01
+SELECT *
+FROM emp
+WHERE EXISTS (SELECT dept.dept_name
+ FROM dept
+ WHERE dept.dept_id > 10
+ OFFSET 2);
+
+-- offset in the exists subquery block with aggregate.
+-- TC.04.02
+SELECT *
+FROM emp
+WHERE EXISTS (SELECT max(dept.dept_id)
+ FROM dept
+ GROUP BY state
+ OFFSET 2);
+
+-- limit in the not exists subquery block.
+-- TC.04.03
+SELECT *
+FROM emp
+WHERE NOT EXISTS (SELECT dept.dept_name
+ FROM dept
+ WHERE dept.dept_id > 100
+ OFFSET 2);
+
+-- limit in the not exists subquery block with aggregates.
+-- TC.04.04
+SELECT *
+FROM emp
+WHERE NOT EXISTS (SELECT max(dept.dept_id)
+ FROM dept
+ WHERE dept.dept_id > 100
+ GROUP BY state
+ OFFSET 2);
diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-limit.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-limit.sql
index 53fc2b8be7501..b9b062a5f7a1d 100644
--- a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-limit.sql
+++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-limit.sql
@@ -100,4 +100,113 @@ WHERE t1d NOT IN (SELECT t2d
LIMIT 1)
GROUP BY t1b
ORDER BY t1b NULLS last
-LIMIT 1;
\ No newline at end of file
+LIMIT 1;
+
+-- LIMIT and OFFSET in parent side
+-- TC 02.01
+SELECT *
+FROM t1
+WHERE t1a IN (SELECT t2a
+ FROM t2
+ WHERE t1d = t2d)
+LIMIT 2
+OFFSET 2;
+
+-- TC 02.02
+SELECT *
+FROM t1
+WHERE t1c IN (SELECT t2c
+ FROM t2
+ WHERE t2b >= 8
+ LIMIT 2
+ OFFSET 2)
+LIMIT 4
+OFFSET 2;
+
+-- TC 02.03
+SELECT Count(DISTINCT( t1a )),
+ t1b
+FROM t1
+WHERE t1d IN (SELECT t2d
+ FROM t2
+ ORDER BY t2c, t2d
+ LIMIT 2)
+GROUP BY t1b
+ORDER BY t1b DESC NULLS FIRST
+LIMIT 1
+OFFSET 1;
+
+-- LIMIT with NOT IN
+-- TC 02.04
+SELECT *
+FROM t1
+WHERE t1b NOT IN (SELECT t2b
+ FROM t2
+ WHERE t2b > 6
+ LIMIT 2
+ OFFSET 2);
+
+-- TC 02.05
+SELECT Count(DISTINCT( t1a )),
+ t1b
+FROM t1
+WHERE t1d NOT IN (SELECT t2d
+ FROM t2
+ ORDER BY t2b DESC nulls first, t2d
+ LIMIT 1
+ OFFSET 1)
+GROUP BY t1b
+ORDER BY t1b NULLS last
+LIMIT 1
+OFFSET 1;
+
+-- OFFSET in parent side
+-- TC 03.01
+SELECT *
+FROM t1
+WHERE t1a IN (SELECT t2a
+ FROM t2
+ WHERE t1d = t2d)
+OFFSET 2;
+
+-- TC 03.02
+SELECT *
+FROM t1
+WHERE t1c IN (SELECT t2c
+ FROM t2
+ WHERE t2b >= 8
+ OFFSET 2)
+OFFSET 4;
+
+-- TC 03.03
+SELECT Count(DISTINCT( t1a )),
+ t1b
+FROM t1
+WHERE t1d IN (SELECT t2d
+ FROM t2
+ ORDER BY t2c, t2d
+ OFFSET 2)
+GROUP BY t1b
+ORDER BY t1b DESC NULLS FIRST
+OFFSET 1;
+
+-- OFFSET with NOT IN
+-- TC 03.04
+SELECT *
+FROM t1
+WHERE t1b NOT IN (SELECT t2b
+ FROM t2
+ WHERE t2b > 6
+ OFFSET 2);
+
+-- TC 03.05
+SELECT Count(DISTINCT( t1a )),
+ t1b
+FROM t1
+WHERE t1d NOT IN (SELECT t2d
+ FROM t2
+ ORDER BY t2b DESC nulls first, t2d
+ OFFSET 1)
+GROUP BY t1b
+ORDER BY t1b NULLS last
+OFFSET 1;
diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/limit.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/limit.sql.out
index 2384010c67b4d..d6526d77d3cad 100644
--- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/limit.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/limit.sql.out
@@ -1,5 +1,5 @@
-- Automatically generated by SQLQueryTestSuite
--- Number of queries: 7
+-- Number of queries: 13
-- !query
@@ -38,6 +38,81 @@ struct
62 633 KCAAAA
+-- !query
+SELECT '' AS three, unique1, unique2, stringu1
+ FROM onek WHERE unique1 > 100
+ ORDER BY unique1 LIMIT 3 OFFSET 20
+-- !query schema
+struct
+-- !query output
+ 121 700 REAAAA
+ 122 519 SEAAAA
+ 123 777 TEAAAA
+
+
+-- !query
+SELECT '' AS zero, unique1, unique2, stringu1
+ FROM onek WHERE unique1 < 50
+ ORDER BY unique1 DESC LIMIT 8 OFFSET 99
+-- !query schema
+struct
+-- !query output
+
+
+
+-- !query
+SELECT '' AS eleven, unique1, unique2, stringu1
+ FROM onek WHERE unique1 < 50
+ ORDER BY unique1 DESC LIMIT 20 OFFSET 39
+-- !query schema
+struct
+-- !query output
+ 10 520 KAAAAA
+ 9 49 JAAAAA
+ 8 653 IAAAAA
+ 7 647 HAAAAA
+ 6 978 GAAAAA
+ 5 541 FAAAAA
+ 4 833 EAAAAA
+ 3 431 DAAAAA
+ 2 326 CAAAAA
+ 1 214 BAAAAA
+ 0 998 AAAAAA
+
+
+-- !query
+SELECT '' AS ten, unique1, unique2, stringu1
+ FROM onek
+ ORDER BY unique1 OFFSET 990
+-- !query schema
+struct
+-- !query output
+ 990 369 CMAAAA
+ 991 426 DMAAAA
+ 992 363 EMAAAA
+ 993 661 FMAAAA
+ 994 695 GMAAAA
+ 995 144 HMAAAA
+ 996 258 IMAAAA
+ 997 21 JMAAAA
+ 998 549 KMAAAA
+ 999 152 LMAAAA
+
+
+-- !query
+SELECT '' AS five, unique1, unique2, stringu1
+ FROM onek
+ ORDER BY unique1 LIMIT 5 OFFSET 900
+-- !query schema
+struct
+-- !query output
+ 900 913 QIAAAA
+ 901 931 RIAAAA
+ 902 702 SIAAAA
+ 903 641 TIAAAA
+ 904 793 UIAAAA
+
+
-- !query
CREATE OR REPLACE TEMPORARY VIEW INT8_TBL AS SELECT * FROM
(VALUES
@@ -62,6 +137,15 @@ org.apache.spark.sql.AnalysisException
The limit expression must evaluate to a constant value, but got CASE WHEN (_nondeterministic < CAST(0.5BD AS DOUBLE)) THEN CAST(NULL AS BIGINT) END
+-- !query
+select * from int8_tbl offset (case when random() < 0.5 then bigint(null) end)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+The offset expression must evaluate to a constant value, but got CASE WHEN (_nondeterministic < CAST(0.5BD AS DOUBLE)) THEN CAST(NULL AS BIGINT) END
+
+
-- !query
DROP VIEW INT8_TBL
-- !query schema
diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/exists-subquery/exists-orderby-limit.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/exists-subquery/exists-orderby-limit.sql.out
index ebd4da6ccbd5d..3442cfd689a2b 100644
--- a/sql/core/src/test/resources/sql-tests/results/subquery/exists-subquery/exists-orderby-limit.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/subquery/exists-subquery/exists-orderby-limit.sql.out
@@ -1,5 +1,5 @@
-- Automatically generated by SQLQueryTestSuite
--- Number of queries: 12
+-- Number of queries: 20
-- !query
@@ -220,3 +220,177 @@ struct
600 emp 6 - no dept 2001-01-01 400.0 100
700 emp 7 2010-01-01 400.0 100
800 emp 8 2016-01-01 150.0 70
+
+
+-- !query
+SELECT *
+FROM emp
+WHERE EXISTS (SELECT dept.dept_name
+ FROM dept
+ WHERE dept.dept_id > 10
+ LIMIT 1
+ OFFSET 2)
+-- !query schema
+struct
+-- !query output
+100 emp 1 2005-01-01 100.0 10
+100 emp 1 2005-01-01 100.0 10
+200 emp 2 2003-01-01 200.0 10
+300 emp 3 2002-01-01 300.0 20
+400 emp 4 2005-01-01 400.0 30
+500 emp 5 2001-01-01 400.0 NULL
+600 emp 6 - no dept 2001-01-01 400.0 100
+700 emp 7 2010-01-01 400.0 100
+800 emp 8 2016-01-01 150.0 70
+
+
+-- !query
+SELECT *
+FROM emp
+WHERE EXISTS (SELECT max(dept.dept_id)
+ FROM dept
+ GROUP BY state
+ LIMIT 1
+ OFFSET 2)
+-- !query schema
+struct
+-- !query output
+100 emp 1 2005-01-01 100.0 10
+100 emp 1 2005-01-01 100.0 10
+200 emp 2 2003-01-01 200.0 10
+300 emp 3 2002-01-01 300.0 20
+400 emp 4 2005-01-01 400.0 30
+500 emp 5 2001-01-01 400.0 NULL
+600 emp 6 - no dept 2001-01-01 400.0 100
+700 emp 7 2010-01-01 400.0 100
+800 emp 8 2016-01-01 150.0 70
+
+
+-- !query
+SELECT *
+FROM emp
+WHERE NOT EXISTS (SELECT dept.dept_name
+ FROM dept
+ WHERE dept.dept_id > 100
+ LIMIT 1
+ OFFSET 2)
+-- !query schema
+struct
+-- !query output
+100 emp 1 2005-01-01 100.0 10
+100 emp 1 2005-01-01 100.0 10
+200 emp 2 2003-01-01 200.0 10
+300 emp 3 2002-01-01 300.0 20
+400 emp 4 2005-01-01 400.0 30
+500 emp 5 2001-01-01 400.0 NULL
+600 emp 6 - no dept 2001-01-01 400.0 100
+700 emp 7 2010-01-01 400.0 100
+800 emp 8 2016-01-01 150.0 70
+
+
+-- !query
+SELECT *
+FROM emp
+WHERE NOT EXISTS (SELECT max(dept.dept_id)
+ FROM dept
+ WHERE dept.dept_id > 100
+ GROUP BY state
+ LIMIT 1
+ OFFSET 2)
+-- !query schema
+struct
+-- !query output
+100 emp 1 2005-01-01 100.0 10
+100 emp 1 2005-01-01 100.0 10
+200 emp 2 2003-01-01 200.0 10
+300 emp 3 2002-01-01 300.0 20
+400 emp 4 2005-01-01 400.0 30
+500 emp 5 2001-01-01 400.0 NULL
+600 emp 6 - no dept 2001-01-01 400.0 100
+700 emp 7 2010-01-01 400.0 100
+800 emp 8 2016-01-01 150.0 70
+
+
+-- !query
+SELECT *
+FROM emp
+WHERE EXISTS (SELECT dept.dept_name
+ FROM dept
+ WHERE dept.dept_id > 10
+ OFFSET 2)
+-- !query schema
+struct
+-- !query output
+100 emp 1 2005-01-01 100.0 10
+100 emp 1 2005-01-01 100.0 10
+200 emp 2 2003-01-01 200.0 10
+300 emp 3 2002-01-01 300.0 20
+400 emp 4 2005-01-01 400.0 30
+500 emp 5 2001-01-01 400.0 NULL
+600 emp 6 - no dept 2001-01-01 400.0 100
+700 emp 7 2010-01-01 400.0 100
+800 emp 8 2016-01-01 150.0 70
+
+
+-- !query
+SELECT *
+FROM emp
+WHERE EXISTS (SELECT max(dept.dept_id)
+ FROM dept
+ GROUP BY state
+ OFFSET 2)
+-- !query schema
+struct
+-- !query output
+100 emp 1 2005-01-01 100.0 10
+100 emp 1 2005-01-01 100.0 10
+200 emp 2 2003-01-01 200.0 10
+300 emp 3 2002-01-01 300.0 20
+400 emp 4 2005-01-01 400.0 30
+500 emp 5 2001-01-01 400.0 NULL
+600 emp 6 - no dept 2001-01-01 400.0 100
+700 emp 7 2010-01-01 400.0 100
+800 emp 8 2016-01-01 150.0 70
+
+
+-- !query
+SELECT *
+FROM emp
+WHERE NOT EXISTS (SELECT dept.dept_name
+ FROM dept
+ WHERE dept.dept_id > 100
+ OFFSET 2)
+-- !query schema
+struct
+-- !query output
+100 emp 1 2005-01-01 100.0 10
+100 emp 1 2005-01-01 100.0 10
+200 emp 2 2003-01-01 200.0 10
+300 emp 3 2002-01-01 300.0 20
+400 emp 4 2005-01-01 400.0 30
+500 emp 5 2001-01-01 400.0 NULL
+600 emp 6 - no dept 2001-01-01 400.0 100
+700 emp 7 2010-01-01 400.0 100
+800 emp 8 2016-01-01 150.0 70
+
+
+-- !query
+SELECT *
+FROM emp
+WHERE NOT EXISTS (SELECT max(dept.dept_id)
+ FROM dept
+ WHERE dept.dept_id > 100
+ GROUP BY state
+ OFFSET 2)
+-- !query schema
+struct
+-- !query output
+100 emp 1 2005-01-01 100.0 10
+100 emp 1 2005-01-01 100.0 10
+200 emp 2 2003-01-01 200.0 10
+300 emp 3 2002-01-01 300.0 20
+400 emp 4 2005-01-01 400.0 30
+500 emp 5 2001-01-01 400.0 NULL
+600 emp 6 - no dept 2001-01-01 400.0 100
+700 emp 7 2010-01-01 400.0 100
+800 emp 8 2016-01-01 150.0 70
diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-limit.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-limit.sql.out
index e24538b9138ba..08f76f0936a56 100644
--- a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-limit.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-limit.sql.out
@@ -1,5 +1,5 @@
-- Automatically generated by SQLQueryTestSuite
--- Number of queries: 8
+-- Number of queries: 18
-- !query
@@ -145,3 +145,175 @@ LIMIT 1
struct
-- !query output
1 6
+
+
+-- !query
+SELECT *
+FROM t1
+WHERE t1a IN (SELECT t2a
+ FROM t2
+ WHERE t1d = t2d)
+LIMIT 2
+OFFSET 2
+-- !query schema
+struct
+-- !query output
+val1e 10 NULL 19 17.0 25.0 2600 2014-05-04 01:01:00 2014-05-04
+val1e 10 NULL 19 17.0 25.0 2600 2014-09-04 01:02:00.001 2014-09-04
+
+
+-- !query
+SELECT *
+FROM t1
+WHERE t1c IN (SELECT t2c
+ FROM t2
+ WHERE t2b >= 8
+ LIMIT 2
+ OFFSET 2)
+LIMIT 4
+OFFSET 2
+-- !query schema
+struct
+-- !query output
+val1d NULL 16 19 17.0 25.0 2600 2014-07-04 01:02:00.001 NULL
+val1d NULL 16 22 17.0 25.0 2600 2014-06-04 01:01:00 NULL
+
+
+-- !query
+SELECT Count(DISTINCT( t1a )),
+ t1b
+FROM t1
+WHERE t1d IN (SELECT t2d
+ FROM t2
+ ORDER BY t2c, t2d
+ LIMIT 2)
+GROUP BY t1b
+ORDER BY t1b DESC NULLS FIRST
+LIMIT 1
+OFFSET 1
+-- !query schema
+struct
+-- !query output
+1 10
+
+
+-- !query
+SELECT *
+FROM t1
+WHERE t1b NOT IN (SELECT t2b
+ FROM t2
+ WHERE t2b > 6
+ LIMIT 2
+ OFFSET 2)
+-- !query schema
+struct
+-- !query output
+val1a 16 12 10 15.0 20.0 2000 2014-07-04 01:01:00 2014-07-04
+val1a 16 12 21 15.0 20.0 2000 2014-06-04 01:02:00.001 2014-06-04
+val1a 6 8 10 15.0 20.0 2000 2014-04-04 01:00:00 2014-04-04
+val1a 6 8 10 15.0 20.0 2000 2014-04-04 01:02:00.001 2014-04-04
+val1d 10 NULL 12 17.0 25.0 2600 2015-05-04 01:01:00 2015-05-04
+val1e 10 NULL 19 17.0 25.0 2600 2014-05-04 01:01:00 2014-05-04
+val1e 10 NULL 19 17.0 25.0 2600 2014-09-04 01:02:00.001 2014-09-04
+val1e 10 NULL 25 17.0 25.0 2600 2014-08-04 01:01:00 2014-08-04
+
+
+-- !query
+SELECT Count(DISTINCT( t1a )),
+ t1b
+FROM t1
+WHERE t1d NOT IN (SELECT t2d
+ FROM t2
+ ORDER BY t2b DESC nulls first, t2d
+ LIMIT 1
+ OFFSET 1)
+GROUP BY t1b
+ORDER BY t1b NULLS last
+LIMIT 1
+OFFSET 1
+-- !query schema
+struct
+-- !query output
+2 8
+
+
+-- !query
+SELECT *
+FROM t1
+WHERE t1a IN (SELECT t2a
+ FROM t2
+ WHERE t1d = t2d)
+OFFSET 2
+-- !query schema
+struct
+-- !query output
+val1e 10 NULL 19 17.0 25.0 2600 2014-05-04 01:01:00 2014-05-04
+val1e 10 NULL 19 17.0 25.0 2600 2014-09-04 01:02:00.001 2014-09-04
+
+
+-- !query
+SELECT *
+FROM t1
+WHERE t1c IN (SELECT t2c
+ FROM t2
+ WHERE t2b >= 8
+ OFFSET 2)
+OFFSET 4
+-- !query schema
+struct
+-- !query output
+val1d NULL 16 19 17.0 25.0 2600 2014-07-04 01:02:00.001 NULL
+val1d NULL 16 22 17.0 25.0 2600 2014-06-04 01:01:00 NULL
+
+
+-- !query
+SELECT Count(DISTINCT( t1a )),
+ t1b
+FROM t1
+WHERE t1d IN (SELECT t2d
+ FROM t2
+ ORDER BY t2c, t2d
+ OFFSET 2)
+GROUP BY t1b
+ORDER BY t1b DESC NULLS FIRST
+OFFSET 1
+-- !query schema
+struct
+-- !query output
+1 10
+2 8
+
+
+-- !query
+SELECT *
+FROM t1
+WHERE t1b NOT IN (SELECT t2b
+ FROM t2
+ WHERE t2b > 6
+ OFFSET 2)
+-- !query schema
+struct
+-- !query output
+val1a 16 12 10 15.0 20.0 2000 2014-07-04 01:01:00 2014-07-04
+val1a 16 12 21 15.0 20.0 2000 2014-06-04 01:02:00.001 2014-06-04
+val1a 6 8 10 15.0 20.0 2000 2014-04-04 01:00:00 2014-04-04
+val1a 6 8 10 15.0 20.0 2000 2014-04-04 01:02:00.001 2014-04-04
+
+
+-- !query
+SELECT Count(DISTINCT( t1a )),
+ t1b
+FROM t1
+WHERE t1d NOT IN (SELECT t2d
+ FROM t2
+ ORDER BY t2b DESC nulls first, t2d
+ OFFSET 1)
+GROUP BY t1b
+ORDER BY t1b NULLS last
+OFFSET 1
+-- !query schema
+struct
+-- !query output
+2 10
+1 16
+1 NULL
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q10.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q10.sf100/explain.txt
index 997ec28f49296..02ff6fcce531d 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q10.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q10.sf100/explain.txt
@@ -262,7 +262,7 @@ Results [14]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, count
(48) TakeOrderedAndProject
Input [14]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cnt1#37, cd_purchase_estimate#28, cnt2#38, cd_credit_rating#29, cnt3#39, cd_dep_count#30, cnt4#40, cd_dep_employed_count#31, cnt5#41, cd_dep_college_count#32, cnt6#42]
-Arguments: 100, 0, [cd_gender#25 ASC NULLS FIRST, cd_marital_status#26 ASC NULLS FIRST, cd_education_status#27 ASC NULLS FIRST, cd_purchase_estimate#28 ASC NULLS FIRST, cd_credit_rating#29 ASC NULLS FIRST, cd_dep_count#30 ASC NULLS FIRST, cd_dep_employed_count#31 ASC NULLS FIRST, cd_dep_college_count#32 ASC NULLS FIRST], [cd_gender#25, cd_marital_status#26, cd_education_status#27, cnt1#37, cd_purchase_estimate#28, cnt2#38, cd_credit_rating#29, cnt3#39, cd_dep_count#30, cnt4#40, cd_dep_employed_count#31, cnt5#41, cd_dep_college_count#32, cnt6#42]
+Arguments: 100, [cd_gender#25 ASC NULLS FIRST, cd_marital_status#26 ASC NULLS FIRST, cd_education_status#27 ASC NULLS FIRST, cd_purchase_estimate#28 ASC NULLS FIRST, cd_credit_rating#29 ASC NULLS FIRST, cd_dep_count#30 ASC NULLS FIRST, cd_dep_employed_count#31 ASC NULLS FIRST, cd_dep_college_count#32 ASC NULLS FIRST], [cd_gender#25, cd_marital_status#26, cd_education_status#27, cnt1#37, cd_purchase_estimate#28, cnt2#38, cd_credit_rating#29, cnt3#39, cd_dep_count#30, cnt4#40, cd_dep_employed_count#31, cnt5#41, cd_dep_college_count#32, cnt6#42]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q10/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q10/explain.txt
index d0fa367ac3724..a565ac5010026 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q10/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q10/explain.txt
@@ -242,7 +242,7 @@ Results [14]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, count
(44) TakeOrderedAndProject
Input [14]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cnt1#36, cd_purchase_estimate#26, cnt2#37, cd_credit_rating#27, cnt3#38, cd_dep_count#28, cnt4#39, cd_dep_employed_count#29, cnt5#40, cd_dep_college_count#30, cnt6#41]
-Arguments: 100, 0, [cd_gender#23 ASC NULLS FIRST, cd_marital_status#24 ASC NULLS FIRST, cd_education_status#25 ASC NULLS FIRST, cd_purchase_estimate#26 ASC NULLS FIRST, cd_credit_rating#27 ASC NULLS FIRST, cd_dep_count#28 ASC NULLS FIRST, cd_dep_employed_count#29 ASC NULLS FIRST, cd_dep_college_count#30 ASC NULLS FIRST], [cd_gender#23, cd_marital_status#24, cd_education_status#25, cnt1#36, cd_purchase_estimate#26, cnt2#37, cd_credit_rating#27, cnt3#38, cd_dep_count#28, cnt4#39, cd_dep_employed_count#29, cnt5#40, cd_dep_college_count#30, cnt6#41]
+Arguments: 100, [cd_gender#23 ASC NULLS FIRST, cd_marital_status#24 ASC NULLS FIRST, cd_education_status#25 ASC NULLS FIRST, cd_purchase_estimate#26 ASC NULLS FIRST, cd_credit_rating#27 ASC NULLS FIRST, cd_dep_count#28 ASC NULLS FIRST, cd_dep_employed_count#29 ASC NULLS FIRST, cd_dep_college_count#30 ASC NULLS FIRST], [cd_gender#23, cd_marital_status#24, cd_education_status#25, cnt1#36, cd_purchase_estimate#26, cnt2#37, cd_credit_rating#27, cnt3#38, cd_dep_count#28, cnt4#39, cd_dep_employed_count#29, cnt5#40, cd_dep_college_count#30, cnt6#41]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q19.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q19.sf100/explain.txt
index 12fec29da2e1c..6b4fce45b44f2 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q19.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q19.sf100/explain.txt
@@ -218,7 +218,7 @@ Results [5]: [i_brand_id#21 AS brand_id#31, i_brand#22 AS brand#32, i_manufact_i
(39) TakeOrderedAndProject
Input [5]: [brand_id#31, brand#32, i_manufact_id#23, i_manufact#24, ext_price#33]
-Arguments: 100, 0, [ext_price#33 DESC NULLS LAST, brand#32 ASC NULLS FIRST, brand_id#31 ASC NULLS FIRST, i_manufact_id#23 ASC NULLS FIRST, i_manufact#24 ASC NULLS FIRST], [brand_id#31, brand#32, i_manufact_id#23, i_manufact#24, ext_price#33]
+Arguments: 100, [ext_price#33 DESC NULLS LAST, brand#32 ASC NULLS FIRST, brand_id#31 ASC NULLS FIRST, i_manufact_id#23 ASC NULLS FIRST, i_manufact#24 ASC NULLS FIRST], [brand_id#31, brand#32, i_manufact_id#23, i_manufact#24, ext_price#33]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q19/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q19/explain.txt
index c70751cf80f87..e58fb8a1e1364 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q19/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q19/explain.txt
@@ -218,5 +218,5 @@ Results [5]: [i_brand_id#11 AS brand_id#30, i_brand#12 AS brand#31, i_manufact_i
(39) TakeOrderedAndProject
Input [5]: [brand_id#30, brand#31, i_manufact_id#13, i_manufact#14, ext_price#32]
-Arguments: 100, 0, [ext_price#32 DESC NULLS LAST, brand#31 ASC NULLS FIRST, brand_id#30 ASC NULLS FIRST, i_manufact_id#13 ASC NULLS FIRST, i_manufact#14 ASC NULLS FIRST], [brand_id#30, brand#31, i_manufact_id#13, i_manufact#14, ext_price#32]
+Arguments: 100, [ext_price#32 DESC NULLS LAST, brand#31 ASC NULLS FIRST, brand_id#30 ASC NULLS FIRST, i_manufact_id#13 ASC NULLS FIRST, i_manufact#14 ASC NULLS FIRST], [brand_id#30, brand#31, i_manufact_id#13, i_manufact#14, ext_price#32]
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q27.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q27.sf100/explain.txt
index b24f36d19dd9b..65572cfd770d0 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q27.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q27.sf100/explain.txt
@@ -404,7 +404,7 @@ Results [7]: [null AS i_item_id#102, null AS s_state#103, 1 AS g_state#104, avg(
(73) TakeOrderedAndProject
Input [7]: [i_item_id#20, s_state#17, g_state#47, agg1#48, agg2#49, agg3#50, agg4#51]
-Arguments: 100, 0, [i_item_id#20 ASC NULLS FIRST, s_state#17 ASC NULLS FIRST], [i_item_id#20, s_state#17, g_state#47, agg1#48, agg2#49, agg3#50, agg4#51]
+Arguments: 100, [i_item_id#20 ASC NULLS FIRST, s_state#17 ASC NULLS FIRST], [i_item_id#20, s_state#17, g_state#47, agg1#48, agg2#49, agg3#50, agg4#51]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q27/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q27/explain.txt
index 5fee0b8192a96..0fe719015aaa3 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q27/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q27/explain.txt
@@ -404,7 +404,7 @@ Results [7]: [null AS i_item_id#102, null AS s_state#103, 1 AS g_state#104, avg(
(73) TakeOrderedAndProject
Input [7]: [i_item_id#20, s_state#17, g_state#47, agg1#48, agg2#49, agg3#50, agg4#51]
-Arguments: 100, 0, [i_item_id#20 ASC NULLS FIRST, s_state#17 ASC NULLS FIRST], [i_item_id#20, s_state#17, g_state#47, agg1#48, agg2#49, agg3#50, agg4#51]
+Arguments: 100, [i_item_id#20 ASC NULLS FIRST, s_state#17 ASC NULLS FIRST], [i_item_id#20, s_state#17, g_state#47, agg1#48, agg2#49, agg3#50, agg4#51]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q3.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q3.sf100/explain.txt
index 6a6ae985d1f65..36258b96121f4 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q3.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q3.sf100/explain.txt
@@ -96,7 +96,7 @@ Results [4]: [d_year#11, i_brand_id#6 AS brand_id#16, i_brand#7 AS brand#17, Mak
(17) TakeOrderedAndProject
Input [4]: [d_year#11, brand_id#16, brand#17, sum_agg#18]
-Arguments: 100, 0, [d_year#11 ASC NULLS FIRST, sum_agg#18 DESC NULLS LAST, brand_id#16 ASC NULLS FIRST], [d_year#11, brand_id#16, brand#17, sum_agg#18]
+Arguments: 100, [d_year#11 ASC NULLS FIRST, sum_agg#18 DESC NULLS LAST, brand_id#16 ASC NULLS FIRST], [d_year#11, brand_id#16, brand#17, sum_agg#18]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q3/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q3/explain.txt
index afce3e4dad180..a60caf19f23df 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q3/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q3/explain.txt
@@ -119,5 +119,5 @@ Results [4]: [d_year#2, i_brand_id#9 AS brand_id#17, i_brand#10 AS brand#18, Mak
(21) TakeOrderedAndProject
Input [4]: [d_year#2, brand_id#17, brand#18, sum_agg#19]
-Arguments: 100, 0, [d_year#2 ASC NULLS FIRST, sum_agg#19 DESC NULLS LAST, brand_id#17 ASC NULLS FIRST], [d_year#2, brand_id#17, brand#18, sum_agg#19]
+Arguments: 100, [d_year#2 ASC NULLS FIRST, sum_agg#19 DESC NULLS LAST, brand_id#17 ASC NULLS FIRST], [d_year#2, brand_id#17, brand#18, sum_agg#19]
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q42.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q42.sf100/explain.txt
index 5f117797460ed..5e83e995a3766 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q42.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q42.sf100/explain.txt
@@ -119,7 +119,7 @@ Results [4]: [d_year#2, i_category_id#10, i_category#11, MakeDecimal(sum(Unscale
(21) TakeOrderedAndProject
Input [4]: [d_year#2, i_category_id#10, i_category#11, sum(ss_ext_sales_price)#18]
-Arguments: 100, 0, [sum(ss_ext_sales_price)#18 DESC NULLS LAST, d_year#2 ASC NULLS FIRST, i_category_id#10 ASC NULLS FIRST, i_category#11 ASC NULLS FIRST], [d_year#2, i_category_id#10, i_category#11, sum(ss_ext_sales_price)#18]
+Arguments: 100, [sum(ss_ext_sales_price)#18 DESC NULLS LAST, d_year#2 ASC NULLS FIRST, i_category_id#10 ASC NULLS FIRST, i_category#11 ASC NULLS FIRST], [d_year#2, i_category_id#10, i_category#11, sum(ss_ext_sales_price)#18]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q42/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q42/explain.txt
index 4fa60d609ecb1..255f39414f908 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q42/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q42/explain.txt
@@ -119,5 +119,5 @@ Results [4]: [d_year#2, i_category_id#9, i_category#10, MakeDecimal(sum(Unscaled
(21) TakeOrderedAndProject
Input [4]: [d_year#2, i_category_id#9, i_category#10, sum(ss_ext_sales_price)#17]
-Arguments: 100, 0, [sum(ss_ext_sales_price)#17 DESC NULLS LAST, d_year#2 ASC NULLS FIRST, i_category_id#9 ASC NULLS FIRST, i_category#10 ASC NULLS FIRST], [d_year#2, i_category_id#9, i_category#10, sum(ss_ext_sales_price)#17]
+Arguments: 100, [sum(ss_ext_sales_price)#17 DESC NULLS LAST, d_year#2 ASC NULLS FIRST, i_category_id#9 ASC NULLS FIRST, i_category#10 ASC NULLS FIRST], [d_year#2, i_category_id#9, i_category#10, sum(ss_ext_sales_price)#17]
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q43.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q43.sf100/explain.txt
index c9861a6680fdf..ec49a329ecded 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q43.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q43.sf100/explain.txt
@@ -119,7 +119,7 @@ Results [9]: [s_store_name#11, s_store_id#10, MakeDecimal(sum(UnscaledValue(CASE
(21) TakeOrderedAndProject
Input [9]: [s_store_name#11, s_store_id#10, sun_sales#36, mon_sales#37, tue_sales#38, wed_sales#39, thu_sales#40, fri_sales#41, sat_sales#42]
-Arguments: 100, 0, [s_store_name#11 ASC NULLS FIRST, s_store_id#10 ASC NULLS FIRST, sun_sales#36 ASC NULLS FIRST, mon_sales#37 ASC NULLS FIRST, tue_sales#38 ASC NULLS FIRST, wed_sales#39 ASC NULLS FIRST, thu_sales#40 ASC NULLS FIRST, fri_sales#41 ASC NULLS FIRST, sat_sales#42 ASC NULLS FIRST], [s_store_name#11, s_store_id#10, sun_sales#36, mon_sales#37, tue_sales#38, wed_sales#39, thu_sales#40, fri_sales#41, sat_sales#42]
+Arguments: 100, [s_store_name#11 ASC NULLS FIRST, s_store_id#10 ASC NULLS FIRST, sun_sales#36 ASC NULLS FIRST, mon_sales#37 ASC NULLS FIRST, tue_sales#38 ASC NULLS FIRST, wed_sales#39 ASC NULLS FIRST, thu_sales#40 ASC NULLS FIRST, fri_sales#41 ASC NULLS FIRST, sat_sales#42 ASC NULLS FIRST], [s_store_name#11, s_store_id#10, sun_sales#36, mon_sales#37, tue_sales#38, wed_sales#39, thu_sales#40, fri_sales#41, sat_sales#42]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q43/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q43/explain.txt
index 38ff8714c9615..2c15f0ebcbd54 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q43/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q43/explain.txt
@@ -119,5 +119,5 @@ Results [9]: [s_store_name#10, s_store_id#9, MakeDecimal(sum(UnscaledValue(CASE
(21) TakeOrderedAndProject
Input [9]: [s_store_name#10, s_store_id#9, sun_sales#35, mon_sales#36, tue_sales#37, wed_sales#38, thu_sales#39, fri_sales#40, sat_sales#41]
-Arguments: 100, 0, [s_store_name#10 ASC NULLS FIRST, s_store_id#9 ASC NULLS FIRST, sun_sales#35 ASC NULLS FIRST, mon_sales#36 ASC NULLS FIRST, tue_sales#37 ASC NULLS FIRST, wed_sales#38 ASC NULLS FIRST, thu_sales#39 ASC NULLS FIRST, fri_sales#40 ASC NULLS FIRST, sat_sales#41 ASC NULLS FIRST], [s_store_name#10, s_store_id#9, sun_sales#35, mon_sales#36, tue_sales#37, wed_sales#38, thu_sales#39, fri_sales#40, sat_sales#41]
+Arguments: 100, [s_store_name#10 ASC NULLS FIRST, s_store_id#9 ASC NULLS FIRST, sun_sales#35 ASC NULLS FIRST, mon_sales#36 ASC NULLS FIRST, tue_sales#37 ASC NULLS FIRST, wed_sales#38 ASC NULLS FIRST, thu_sales#39 ASC NULLS FIRST, fri_sales#40 ASC NULLS FIRST, sat_sales#41 ASC NULLS FIRST], [s_store_name#10, s_store_id#9, sun_sales#35, mon_sales#36, tue_sales#37, wed_sales#38, thu_sales#39, fri_sales#40, sat_sales#41]
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q46.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q46.sf100/explain.txt
index be323349b900d..4ea916a38b26b 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q46.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q46.sf100/explain.txt
@@ -255,7 +255,7 @@ Input [9]: [ss_ticket_number#5, bought_city#28, amt#29, profit#30, c_current_add
(47) TakeOrderedAndProject
Input [7]: [c_last_name#35, c_first_name#34, ca_city#39, bought_city#28, ss_ticket_number#5, amt#29, profit#30]
-Arguments: 100, 0, [c_last_name#35 ASC NULLS FIRST, c_first_name#34 ASC NULLS FIRST, ca_city#39 ASC NULLS FIRST, bought_city#28 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#35, c_first_name#34, ca_city#39, bought_city#28, ss_ticket_number#5, amt#29, profit#30]
+Arguments: 100, [c_last_name#35 ASC NULLS FIRST, c_first_name#34 ASC NULLS FIRST, ca_city#39 ASC NULLS FIRST, bought_city#28 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#35, c_first_name#34, ca_city#39, bought_city#28, ss_ticket_number#5, amt#29, profit#30]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q46/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q46/explain.txt
index 22b8206c0fd24..5d36c1d7cd2f5 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q46/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q46/explain.txt
@@ -215,7 +215,7 @@ Input [9]: [ss_ticket_number#5, bought_city#28, amt#29, profit#30, c_current_add
(39) TakeOrderedAndProject
Input [7]: [c_last_name#34, c_first_name#33, ca_city#37, bought_city#28, ss_ticket_number#5, amt#29, profit#30]
-Arguments: 100, 0, [c_last_name#34 ASC NULLS FIRST, c_first_name#33 ASC NULLS FIRST, ca_city#37 ASC NULLS FIRST, bought_city#28 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#34, c_first_name#33, ca_city#37, bought_city#28, ss_ticket_number#5, amt#29, profit#30]
+Arguments: 100, [c_last_name#34 ASC NULLS FIRST, c_first_name#33 ASC NULLS FIRST, ca_city#37 ASC NULLS FIRST, bought_city#28 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#34, c_first_name#33, ca_city#37, bought_city#28, ss_ticket_number#5, amt#29, profit#30]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q52.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q52.sf100/explain.txt
index 90f1eb77ec15f..02f680fc0dd1a 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q52.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q52.sf100/explain.txt
@@ -119,7 +119,7 @@ Results [4]: [d_year#2, i_brand_id#10 AS brand_id#18, i_brand#11 AS brand#19, Ma
(21) TakeOrderedAndProject
Input [4]: [d_year#2, brand_id#18, brand#19, ext_price#20]
-Arguments: 100, 0, [d_year#2 ASC NULLS FIRST, ext_price#20 DESC NULLS LAST, brand_id#18 ASC NULLS FIRST], [d_year#2, brand_id#18, brand#19, ext_price#20]
+Arguments: 100, [d_year#2 ASC NULLS FIRST, ext_price#20 DESC NULLS LAST, brand_id#18 ASC NULLS FIRST], [d_year#2, brand_id#18, brand#19, ext_price#20]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q52/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q52/explain.txt
index 77cf242e5b169..8d081d60f9e85 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q52/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q52/explain.txt
@@ -119,5 +119,5 @@ Results [4]: [d_year#2, i_brand_id#9 AS brand_id#17, i_brand#10 AS brand#18, Mak
(21) TakeOrderedAndProject
Input [4]: [d_year#2, brand_id#17, brand#18, ext_price#19]
-Arguments: 100, 0, [d_year#2 ASC NULLS FIRST, ext_price#19 DESC NULLS LAST, brand_id#17 ASC NULLS FIRST], [d_year#2, brand_id#17, brand#18, ext_price#19]
+Arguments: 100, [d_year#2 ASC NULLS FIRST, ext_price#19 DESC NULLS LAST, brand_id#17 ASC NULLS FIRST], [d_year#2, brand_id#17, brand#18, ext_price#19]
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q53.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q53.sf100/explain.txt
index 09ca40b47f838..89434b0b06e8f 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q53.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q53.sf100/explain.txt
@@ -154,7 +154,7 @@ Input [4]: [i_manufact_id#5, sum_sales#24, _w0#25, avg_quarterly_sales#27]
(28) TakeOrderedAndProject
Input [3]: [i_manufact_id#5, sum_sales#24, avg_quarterly_sales#27]
-Arguments: 100, 0, [avg_quarterly_sales#27 ASC NULLS FIRST, sum_sales#24 ASC NULLS FIRST, i_manufact_id#5 ASC NULLS FIRST], [i_manufact_id#5, sum_sales#24, avg_quarterly_sales#27]
+Arguments: 100, [avg_quarterly_sales#27 ASC NULLS FIRST, sum_sales#24 ASC NULLS FIRST, i_manufact_id#5 ASC NULLS FIRST], [i_manufact_id#5, sum_sales#24, avg_quarterly_sales#27]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q53/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q53/explain.txt
index 849f576980648..4ad6ced134b5a 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q53/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q53/explain.txt
@@ -154,7 +154,7 @@ Input [4]: [i_manufact_id#5, sum_sales#24, _w0#25, avg_quarterly_sales#27]
(28) TakeOrderedAndProject
Input [3]: [i_manufact_id#5, sum_sales#24, avg_quarterly_sales#27]
-Arguments: 100, 0, [avg_quarterly_sales#27 ASC NULLS FIRST, sum_sales#24 ASC NULLS FIRST, i_manufact_id#5 ASC NULLS FIRST], [i_manufact_id#5, sum_sales#24, avg_quarterly_sales#27]
+Arguments: 100, [avg_quarterly_sales#27 ASC NULLS FIRST, sum_sales#24 ASC NULLS FIRST, i_manufact_id#5 ASC NULLS FIRST], [i_manufact_id#5, sum_sales#24, avg_quarterly_sales#27]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q55.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q55.sf100/explain.txt
index fb8b7c9004f03..df4d94f40bc16 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q55.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q55.sf100/explain.txt
@@ -119,7 +119,7 @@ Results [3]: [i_brand_id#10 AS brand_id#18, i_brand#11 AS brand#19, MakeDecimal(
(21) TakeOrderedAndProject
Input [3]: [brand_id#18, brand#19, ext_price#20]
-Arguments: 100, 0, [ext_price#20 DESC NULLS LAST, brand_id#18 ASC NULLS FIRST], [brand_id#18, brand#19, ext_price#20]
+Arguments: 100, [ext_price#20 DESC NULLS LAST, brand_id#18 ASC NULLS FIRST], [brand_id#18, brand#19, ext_price#20]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q55/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q55/explain.txt
index b0c67cb339a73..a10fc3256db33 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q55/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q55/explain.txt
@@ -119,5 +119,5 @@ Results [3]: [i_brand_id#9 AS brand_id#17, i_brand#10 AS brand#18, MakeDecimal(s
(21) TakeOrderedAndProject
Input [3]: [brand_id#17, brand#18, ext_price#19]
-Arguments: 100, 0, [ext_price#19 DESC NULLS LAST, brand_id#17 ASC NULLS FIRST], [brand_id#17, brand#18, ext_price#19]
+Arguments: 100, [ext_price#19 DESC NULLS LAST, brand_id#17 ASC NULLS FIRST], [brand_id#17, brand#18, ext_price#19]
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q59.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q59.sf100/explain.txt
index ec154a3a89df6..d500ebfb18c8f 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q59.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q59.sf100/explain.txt
@@ -288,5 +288,5 @@ Input [18]: [s_store_name1#44, d_week_seq1#45, s_store_id1#46, sun_sales1#47, mo
(51) TakeOrderedAndProject
Input [10]: [s_store_name1#44, s_store_id1#46, d_week_seq1#45, (sun_sales1 / sun_sales2)#82, (mon_sales1 / mon_sales2)#83, (tue_sales1 / tue_sales1)#84, (wed_sales1 / wed_sales2)#85, (thu_sales1 / thu_sales2)#86, (fri_sales1 / fri_sales2)#87, (sat_sales1 / sat_sales2)#88]
-Arguments: 100, 0, [s_store_name1#44 ASC NULLS FIRST, s_store_id1#46 ASC NULLS FIRST, d_week_seq1#45 ASC NULLS FIRST], [s_store_name1#44, s_store_id1#46, d_week_seq1#45, (sun_sales1 / sun_sales2)#82, (mon_sales1 / mon_sales2)#83, (tue_sales1 / tue_sales1)#84, (wed_sales1 / wed_sales2)#85, (thu_sales1 / thu_sales2)#86, (fri_sales1 / fri_sales2)#87, (sat_sales1 / sat_sales2)#88]
+Arguments: 100, [s_store_name1#44 ASC NULLS FIRST, s_store_id1#46 ASC NULLS FIRST, d_week_seq1#45 ASC NULLS FIRST], [s_store_name1#44, s_store_id1#46, d_week_seq1#45, (sun_sales1 / sun_sales2)#82, (mon_sales1 / mon_sales2)#83, (tue_sales1 / tue_sales1)#84, (wed_sales1 / wed_sales2)#85, (thu_sales1 / thu_sales2)#86, (fri_sales1 / fri_sales2)#87, (sat_sales1 / sat_sales2)#88]
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q59/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q59/explain.txt
index ec154a3a89df6..d500ebfb18c8f 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q59/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q59/explain.txt
@@ -288,5 +288,5 @@ Input [18]: [s_store_name1#44, d_week_seq1#45, s_store_id1#46, sun_sales1#47, mo
(51) TakeOrderedAndProject
Input [10]: [s_store_name1#44, s_store_id1#46, d_week_seq1#45, (sun_sales1 / sun_sales2)#82, (mon_sales1 / mon_sales2)#83, (tue_sales1 / tue_sales1)#84, (wed_sales1 / wed_sales2)#85, (thu_sales1 / thu_sales2)#86, (fri_sales1 / fri_sales2)#87, (sat_sales1 / sat_sales2)#88]
-Arguments: 100, 0, [s_store_name1#44 ASC NULLS FIRST, s_store_id1#46 ASC NULLS FIRST, d_week_seq1#45 ASC NULLS FIRST], [s_store_name1#44, s_store_id1#46, d_week_seq1#45, (sun_sales1 / sun_sales2)#82, (mon_sales1 / mon_sales2)#83, (tue_sales1 / tue_sales1)#84, (wed_sales1 / wed_sales2)#85, (thu_sales1 / thu_sales2)#86, (fri_sales1 / fri_sales2)#87, (sat_sales1 / sat_sales2)#88]
+Arguments: 100, [s_store_name1#44 ASC NULLS FIRST, s_store_id1#46 ASC NULLS FIRST, d_week_seq1#45 ASC NULLS FIRST], [s_store_name1#44, s_store_id1#46, d_week_seq1#45, (sun_sales1 / sun_sales2)#82, (mon_sales1 / mon_sales2)#83, (tue_sales1 / tue_sales1)#84, (wed_sales1 / wed_sales2)#85, (thu_sales1 / thu_sales2)#86, (fri_sales1 / fri_sales2)#87, (sat_sales1 / sat_sales2)#88]
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q63.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q63.sf100/explain.txt
index bda0b173dd4bb..35e087b5f1087 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q63.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q63.sf100/explain.txt
@@ -154,7 +154,7 @@ Input [4]: [i_manager_id#5, sum_sales#24, _w0#25, avg_monthly_sales#27]
(28) TakeOrderedAndProject
Input [3]: [i_manager_id#5, sum_sales#24, avg_monthly_sales#27]
-Arguments: 100, 0, [i_manager_id#5 ASC NULLS FIRST, avg_monthly_sales#27 ASC NULLS FIRST, sum_sales#24 ASC NULLS FIRST], [i_manager_id#5, sum_sales#24, avg_monthly_sales#27]
+Arguments: 100, [i_manager_id#5 ASC NULLS FIRST, avg_monthly_sales#27 ASC NULLS FIRST, sum_sales#24 ASC NULLS FIRST], [i_manager_id#5, sum_sales#24, avg_monthly_sales#27]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q63/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q63/explain.txt
index fa5e6a459951d..94181f55279a8 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q63/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q63/explain.txt
@@ -154,7 +154,7 @@ Input [4]: [i_manager_id#5, sum_sales#24, _w0#25, avg_monthly_sales#27]
(28) TakeOrderedAndProject
Input [3]: [i_manager_id#5, sum_sales#24, avg_monthly_sales#27]
-Arguments: 100, 0, [i_manager_id#5 ASC NULLS FIRST, avg_monthly_sales#27 ASC NULLS FIRST, sum_sales#24 ASC NULLS FIRST], [i_manager_id#5, sum_sales#24, avg_monthly_sales#27]
+Arguments: 100, [i_manager_id#5 ASC NULLS FIRST, avg_monthly_sales#27 ASC NULLS FIRST, sum_sales#24 ASC NULLS FIRST], [i_manager_id#5, sum_sales#24, avg_monthly_sales#27]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65.sf100/explain.txt
index 95d9032416e17..b0db241fd5061 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65.sf100/explain.txt
@@ -225,7 +225,7 @@ Input [8]: [ss_item_sk#1, revenue#11, s_store_name#32, i_item_sk#34, i_item_desc
(39) TakeOrderedAndProject
Input [6]: [s_store_name#32, i_item_desc#35, revenue#11, i_current_price#36, i_wholesale_cost#37, i_brand#38]
-Arguments: 100, 0, [s_store_name#32 ASC NULLS FIRST, i_item_desc#35 ASC NULLS FIRST], [s_store_name#32, i_item_desc#35, revenue#11, i_current_price#36, i_wholesale_cost#37, i_brand#38]
+Arguments: 100, [s_store_name#32 ASC NULLS FIRST, i_item_desc#35 ASC NULLS FIRST], [s_store_name#32, i_item_desc#35, revenue#11, i_current_price#36, i_wholesale_cost#37, i_brand#38]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65/explain.txt
index 1f9a63db2ff89..84dd6c4f3a5d9 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65/explain.txt
@@ -225,7 +225,7 @@ Input [9]: [s_store_name#2, ss_store_sk#4, revenue#13, i_item_desc#16, i_current
(39) TakeOrderedAndProject
Input [6]: [s_store_name#2, i_item_desc#16, revenue#13, i_current_price#17, i_wholesale_cost#18, i_brand#19]
-Arguments: 100, 0, [s_store_name#2 ASC NULLS FIRST, i_item_desc#16 ASC NULLS FIRST], [s_store_name#2, i_item_desc#16, revenue#13, i_current_price#17, i_wholesale_cost#18, i_brand#19]
+Arguments: 100, [s_store_name#2 ASC NULLS FIRST, i_item_desc#16 ASC NULLS FIRST], [s_store_name#2, i_item_desc#16, revenue#13, i_current_price#17, i_wholesale_cost#18, i_brand#19]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q68.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q68.sf100/explain.txt
index c6f480e11efe8..020f7344233b1 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q68.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q68.sf100/explain.txt
@@ -248,7 +248,7 @@ Input [10]: [ss_ticket_number#5, bought_city#32, extended_price#33, list_price#3
(45) TakeOrderedAndProject
Input [8]: [c_last_name#40, c_first_name#39, ca_city#43, bought_city#32, ss_ticket_number#5, extended_price#33, extended_tax#35, list_price#34]
-Arguments: 100, 0, [c_last_name#40 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#40, c_first_name#39, ca_city#43, bought_city#32, ss_ticket_number#5, extended_price#33, extended_tax#35, list_price#34]
+Arguments: 100, [c_last_name#40 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#40, c_first_name#39, ca_city#43, bought_city#32, ss_ticket_number#5, extended_price#33, extended_tax#35, list_price#34]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q68/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q68/explain.txt
index 2f07a8ae8f6ee..63f007cf1205e 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q68/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q68/explain.txt
@@ -215,7 +215,7 @@ Input [10]: [ss_ticket_number#5, bought_city#32, extended_price#33, list_price#3
(39) TakeOrderedAndProject
Input [8]: [c_last_name#39, c_first_name#38, ca_city#42, bought_city#32, ss_ticket_number#5, extended_price#33, extended_tax#35, list_price#34]
-Arguments: 100, 0, [c_last_name#39 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#39, c_first_name#38, ca_city#42, bought_city#32, ss_ticket_number#5, extended_price#33, extended_tax#35, list_price#34]
+Arguments: 100, [c_last_name#39 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#39, c_first_name#38, ca_city#42, bought_city#32, ss_ticket_number#5, extended_price#33, extended_tax#35, list_price#34]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q7.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q7.sf100/explain.txt
index 9763baa606a58..adbc64c43ff2f 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q7.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q7.sf100/explain.txt
@@ -167,7 +167,7 @@ Results [5]: [i_item_id#21, avg(ss_quantity#4)#40 AS agg1#44, cast((avg(Unscaled
(30) TakeOrderedAndProject
Input [5]: [i_item_id#21, agg1#44, agg2#45, agg3#46, agg4#47]
-Arguments: 100, 0, [i_item_id#21 ASC NULLS FIRST], [i_item_id#21, agg1#44, agg2#45, agg3#46, agg4#47]
+Arguments: 100, [i_item_id#21 ASC NULLS FIRST], [i_item_id#21, agg1#44, agg2#45, agg3#46, agg4#47]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q7/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q7/explain.txt
index 3730b1b4d7314..afcfd4816dcad 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q7/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q7/explain.txt
@@ -167,7 +167,7 @@ Results [5]: [i_item_id#17, avg(ss_quantity#4)#40 AS agg1#44, cast((avg(Unscaled
(30) TakeOrderedAndProject
Input [5]: [i_item_id#17, agg1#44, agg2#45, agg3#46, agg4#47]
-Arguments: 100, 0, [i_item_id#17 ASC NULLS FIRST], [i_item_id#17, agg1#44, agg2#45, agg3#46, agg4#47]
+Arguments: 100, [i_item_id#17 ASC NULLS FIRST], [i_item_id#17, agg1#44, agg2#45, agg3#46, agg4#47]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q79.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q79.sf100/explain.txt
index 54d0f801b2a29..739c29798ad6e 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q79.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q79.sf100/explain.txt
@@ -182,7 +182,7 @@ Input [8]: [ss_ticket_number#5, ss_customer_sk#1, s_city#17, amt#26, profit#27,
(33) TakeOrderedAndProject
Input [7]: [c_last_name#31, c_first_name#30, substr(s_city, 1, 30)#33, ss_ticket_number#5, amt#26, profit#27, s_city#17]
-Arguments: 100, 0, [c_last_name#31 ASC NULLS FIRST, c_first_name#30 ASC NULLS FIRST, substr(s_city#17, 1, 30) ASC NULLS FIRST, profit#27 ASC NULLS FIRST], [c_last_name#31, c_first_name#30, substr(s_city, 1, 30)#33, ss_ticket_number#5, amt#26, profit#27]
+Arguments: 100, [c_last_name#31 ASC NULLS FIRST, c_first_name#30 ASC NULLS FIRST, substr(s_city#17, 1, 30) ASC NULLS FIRST, profit#27 ASC NULLS FIRST], [c_last_name#31, c_first_name#30, substr(s_city, 1, 30)#33, ss_ticket_number#5, amt#26, profit#27]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q79/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q79/explain.txt
index 22b1ccc37b18d..b00a5500d7c9a 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q79/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q79/explain.txt
@@ -167,7 +167,7 @@ Input [8]: [ss_ticket_number#5, ss_customer_sk#1, s_city#13, amt#26, profit#27,
(30) TakeOrderedAndProject
Input [7]: [c_last_name#30, c_first_name#29, substr(s_city, 1, 30)#32, ss_ticket_number#5, amt#26, profit#27, s_city#13]
-Arguments: 100, 0, [c_last_name#30 ASC NULLS FIRST, c_first_name#29 ASC NULLS FIRST, substr(s_city#13, 1, 30) ASC NULLS FIRST, profit#27 ASC NULLS FIRST], [c_last_name#30, c_first_name#29, substr(s_city, 1, 30)#32, ss_ticket_number#5, amt#26, profit#27]
+Arguments: 100, [c_last_name#30 ASC NULLS FIRST, c_first_name#29 ASC NULLS FIRST, substr(s_city#13, 1, 30) ASC NULLS FIRST, profit#27 ASC NULLS FIRST], [c_last_name#30, c_first_name#29, substr(s_city, 1, 30)#32, ss_ticket_number#5, amt#26, profit#27]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q89.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q89.sf100/explain.txt
index 602ba9adfb128..2bee1ed53e5db 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q89.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q89.sf100/explain.txt
@@ -149,7 +149,7 @@ Input [9]: [i_category#15, i_class#14, i_brand#13, s_store_name#9, s_company_nam
(27) TakeOrderedAndProject
Input [8]: [i_category#15, i_class#14, i_brand#13, s_store_name#9, s_company_name#10, d_moy#7, sum_sales#21, avg_monthly_sales#24]
-Arguments: 100, 0, [(sum_sales#21 - avg_monthly_sales#24) ASC NULLS FIRST, s_store_name#9 ASC NULLS FIRST], [i_category#15, i_class#14, i_brand#13, s_store_name#9, s_company_name#10, d_moy#7, sum_sales#21, avg_monthly_sales#24]
+Arguments: 100, [(sum_sales#21 - avg_monthly_sales#24) ASC NULLS FIRST, s_store_name#9 ASC NULLS FIRST], [i_category#15, i_class#14, i_brand#13, s_store_name#9, s_company_name#10, d_moy#7, sum_sales#21, avg_monthly_sales#24]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q89/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q89/explain.txt
index 326eea6affdbf..ca6080d82947b 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q89/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q89/explain.txt
@@ -149,7 +149,7 @@ Input [9]: [i_category#4, i_class#3, i_brand#2, s_store_name#14, s_company_name#
(27) TakeOrderedAndProject
Input [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#14, s_company_name#15, d_moy#12, sum_sales#21, avg_monthly_sales#24]
-Arguments: 100, 0, [(sum_sales#21 - avg_monthly_sales#24) ASC NULLS FIRST, s_store_name#14 ASC NULLS FIRST], [i_category#4, i_class#3, i_brand#2, s_store_name#14, s_company_name#15, d_moy#12, sum_sales#21, avg_monthly_sales#24]
+Arguments: 100, [(sum_sales#21 - avg_monthly_sales#24) ASC NULLS FIRST, s_store_name#14 ASC NULLS FIRST], [i_category#4, i_class#3, i_brand#2, s_store_name#14, s_company_name#15, d_moy#12, sum_sales#21, avg_monthly_sales#24]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.sf100/explain.txt
index 7c568cd50bd07..a2735ac0e5047 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.sf100/explain.txt
@@ -245,7 +245,7 @@ Input [3]: [ctr_customer_sk#11, c_customer_sk#30, c_customer_id#31]
(43) TakeOrderedAndProject
Input [1]: [c_customer_id#31]
-Arguments: 100, 0, [c_customer_id#31 ASC NULLS FIRST], [c_customer_id#31]
+Arguments: 100, [c_customer_id#31 ASC NULLS FIRST], [c_customer_id#31]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/explain.txt
index 54ea545fc5935..0f43892ddda6d 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/explain.txt
@@ -230,7 +230,7 @@ Input [3]: [ctr_customer_sk#11, c_customer_sk#29, c_customer_id#30]
(40) TakeOrderedAndProject
Input [1]: [c_customer_id#30]
-Arguments: 100, 0, [c_customer_id#30 ASC NULLS FIRST], [c_customer_id#30]
+Arguments: 100, [c_customer_id#30 ASC NULLS FIRST], [c_customer_id#30]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.sf100/explain.txt
index 8e31e1adb1d89..1337d5ed266a3 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.sf100/explain.txt
@@ -277,7 +277,7 @@ Results [14]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, count
(51) TakeOrderedAndProject
Input [14]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cnt1#38, cd_purchase_estimate#28, cnt2#39, cd_credit_rating#29, cnt3#40, cd_dep_count#30, cnt4#41, cd_dep_employed_count#31, cnt5#42, cd_dep_college_count#32, cnt6#43]
-Arguments: 100, 0, [cd_gender#25 ASC NULLS FIRST, cd_marital_status#26 ASC NULLS FIRST, cd_education_status#27 ASC NULLS FIRST, cd_purchase_estimate#28 ASC NULLS FIRST, cd_credit_rating#29 ASC NULLS FIRST, cd_dep_count#30 ASC NULLS FIRST, cd_dep_employed_count#31 ASC NULLS FIRST, cd_dep_college_count#32 ASC NULLS FIRST], [cd_gender#25, cd_marital_status#26, cd_education_status#27, cnt1#38, cd_purchase_estimate#28, cnt2#39, cd_credit_rating#29, cnt3#40, cd_dep_count#30, cnt4#41, cd_dep_employed_count#31, cnt5#42, cd_dep_college_count#32, cnt6#43]
+Arguments: 100, [cd_gender#25 ASC NULLS FIRST, cd_marital_status#26 ASC NULLS FIRST, cd_education_status#27 ASC NULLS FIRST, cd_purchase_estimate#28 ASC NULLS FIRST, cd_credit_rating#29 ASC NULLS FIRST, cd_dep_count#30 ASC NULLS FIRST, cd_dep_employed_count#31 ASC NULLS FIRST, cd_dep_college_count#32 ASC NULLS FIRST], [cd_gender#25, cd_marital_status#26, cd_education_status#27, cnt1#38, cd_purchase_estimate#28, cnt2#39, cd_credit_rating#29, cnt3#40, cd_dep_count#30, cnt4#41, cd_dep_employed_count#31, cnt5#42, cd_dep_college_count#32, cnt6#43]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/explain.txt
index bde8bf2fa7f0f..80c23b3f113a0 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/explain.txt
@@ -237,7 +237,7 @@ Results [14]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, count
(43) TakeOrderedAndProject
Input [14]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cnt1#36, cd_purchase_estimate#26, cnt2#37, cd_credit_rating#27, cnt3#38, cd_dep_count#28, cnt4#39, cd_dep_employed_count#29, cnt5#40, cd_dep_college_count#30, cnt6#41]
-Arguments: 100, 0, [cd_gender#23 ASC NULLS FIRST, cd_marital_status#24 ASC NULLS FIRST, cd_education_status#25 ASC NULLS FIRST, cd_purchase_estimate#26 ASC NULLS FIRST, cd_credit_rating#27 ASC NULLS FIRST, cd_dep_count#28 ASC NULLS FIRST, cd_dep_employed_count#29 ASC NULLS FIRST, cd_dep_college_count#30 ASC NULLS FIRST], [cd_gender#23, cd_marital_status#24, cd_education_status#25, cnt1#36, cd_purchase_estimate#26, cnt2#37, cd_credit_rating#27, cnt3#38, cd_dep_count#28, cnt4#39, cd_dep_employed_count#29, cnt5#40, cd_dep_college_count#30, cnt6#41]
+Arguments: 100, [cd_gender#23 ASC NULLS FIRST, cd_marital_status#24 ASC NULLS FIRST, cd_education_status#25 ASC NULLS FIRST, cd_purchase_estimate#26 ASC NULLS FIRST, cd_credit_rating#27 ASC NULLS FIRST, cd_dep_count#28 ASC NULLS FIRST, cd_dep_employed_count#29 ASC NULLS FIRST, cd_dep_college_count#30 ASC NULLS FIRST], [cd_gender#23, cd_marital_status#24, cd_education_status#25, cnt1#36, cd_purchase_estimate#26, cnt2#37, cd_credit_rating#27, cnt3#38, cd_dep_count#28, cnt4#39, cd_dep_employed_count#29, cnt5#40, cd_dep_college_count#30, cnt6#41]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.sf100/explain.txt
index 9592ea1f23441..d186a969291a7 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.sf100/explain.txt
@@ -446,7 +446,7 @@ Input [7]: [customer_id#22, year_total#23, customer_preferred_cust_flag#45, year
(81) TakeOrderedAndProject
Input [1]: [customer_preferred_cust_flag#45]
-Arguments: 100, 0, [customer_preferred_cust_flag#45 ASC NULLS FIRST], [customer_preferred_cust_flag#45]
+Arguments: 100, [customer_preferred_cust_flag#45 ASC NULLS FIRST], [customer_preferred_cust_flag#45]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/explain.txt
index 2beb7c142116b..0bff392494cd2 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/explain.txt
@@ -415,7 +415,7 @@ Input [7]: [customer_id#21, year_total#22, customer_preferred_cust_flag#43, year
(73) TakeOrderedAndProject
Input [1]: [customer_preferred_cust_flag#43]
-Arguments: 100, 0, [customer_preferred_cust_flag#43 ASC NULLS FIRST], [customer_preferred_cust_flag#43]
+Arguments: 100, [customer_preferred_cust_flag#43 ASC NULLS FIRST], [customer_preferred_cust_flag#43]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.sf100/explain.txt
index b83c1717a52aa..58dc9e1340829 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.sf100/explain.txt
@@ -126,7 +126,7 @@ Input [9]: [i_item_desc#8, i_category#11, i_class#10, i_current_price#9, itemrev
(23) TakeOrderedAndProject
Input [7]: [i_item_desc#8, i_category#11, i_class#10, i_current_price#9, itemrevenue#18, revenueratio#23, i_item_id#7]
-Arguments: 100, 0, [i_category#11 ASC NULLS FIRST, i_class#10 ASC NULLS FIRST, i_item_id#7 ASC NULLS FIRST, i_item_desc#8 ASC NULLS FIRST, revenueratio#23 ASC NULLS FIRST], [i_item_desc#8, i_category#11, i_class#10, i_current_price#9, itemrevenue#18, revenueratio#23]
+Arguments: 100, [i_category#11 ASC NULLS FIRST, i_class#10 ASC NULLS FIRST, i_item_id#7 ASC NULLS FIRST, i_item_desc#8 ASC NULLS FIRST, revenueratio#23 ASC NULLS FIRST], [i_item_desc#8, i_category#11, i_class#10, i_current_price#9, itemrevenue#18, revenueratio#23]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/explain.txt
index 7144c9152e8aa..d10ab7d87cf42 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/explain.txt
@@ -111,7 +111,7 @@ Input [9]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemreve
(20) TakeOrderedAndProject
Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#17, revenueratio#22, i_item_id#6]
-Arguments: 100, 0, [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#22 ASC NULLS FIRST], [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#17, revenueratio#22]
+Arguments: 100, [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#22 ASC NULLS FIRST], [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#17, revenueratio#22]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/explain.txt
index 7aeb27494c408..cb463b13375e2 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/explain.txt
@@ -696,7 +696,7 @@ Results [6]: [channel#112, i_brand_id#113, i_class_id#114, i_category_id#115, su
(126) TakeOrderedAndProject
Input [6]: [channel#112, i_brand_id#113, i_class_id#114, i_category_id#115, sum(sales)#126, sum(number_sales)#127]
-Arguments: 100, 0, [channel#112 ASC NULLS FIRST, i_brand_id#113 ASC NULLS FIRST, i_class_id#114 ASC NULLS FIRST, i_category_id#115 ASC NULLS FIRST], [channel#112, i_brand_id#113, i_class_id#114, i_category_id#115, sum(sales)#126, sum(number_sales)#127]
+Arguments: 100, [channel#112 ASC NULLS FIRST, i_brand_id#113 ASC NULLS FIRST, i_class_id#114 ASC NULLS FIRST, i_category_id#115 ASC NULLS FIRST], [channel#112, i_brand_id#113, i_class_id#114, i_category_id#115, sum(sales)#126, sum(number_sales)#127]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt
index ae467fe5ecfa6..4366f2c5d99d3 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt
@@ -601,7 +601,7 @@ Results [6]: [channel#105, i_brand_id#106, i_class_id#107, i_category_id#108, su
(107) TakeOrderedAndProject
Input [6]: [channel#105, i_brand_id#106, i_class_id#107, i_category_id#108, sum(sales)#119, sum(number_sales)#120]
-Arguments: 100, 0, [channel#105 ASC NULLS FIRST, i_brand_id#106 ASC NULLS FIRST, i_class_id#107 ASC NULLS FIRST, i_category_id#108 ASC NULLS FIRST], [channel#105, i_brand_id#106, i_class_id#107, i_category_id#108, sum(sales)#119, sum(number_sales)#120]
+Arguments: 100, [channel#105 ASC NULLS FIRST, i_brand_id#106 ASC NULLS FIRST, i_class_id#107 ASC NULLS FIRST, i_category_id#108 ASC NULLS FIRST], [channel#105, i_brand_id#106, i_class_id#107, i_category_id#108, sum(sales)#119, sum(number_sales)#120]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/explain.txt
index e41baab617b1b..188a166dbf9c1 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/explain.txt
@@ -564,7 +564,7 @@ Join condition: None
(102) TakeOrderedAndProject
Input [12]: [channel#63, i_brand_id#49, i_class_id#50, i_category_id#51, sales#64, number_sales#65, channel#88, i_brand_id#76, i_class_id#77, i_category_id#78, sales#89, number_sales#90]
-Arguments: 100, 0, [i_brand_id#49 ASC NULLS FIRST, i_class_id#50 ASC NULLS FIRST, i_category_id#51 ASC NULLS FIRST], [channel#63, i_brand_id#49, i_class_id#50, i_category_id#51, sales#64, number_sales#65, channel#88, i_brand_id#76, i_class_id#77, i_category_id#78, sales#89, number_sales#90]
+Arguments: 100, [i_brand_id#49 ASC NULLS FIRST, i_class_id#50 ASC NULLS FIRST, i_category_id#51 ASC NULLS FIRST], [channel#63, i_brand_id#49, i_class_id#50, i_category_id#51, sales#64, number_sales#65, channel#88, i_brand_id#76, i_class_id#77, i_category_id#78, sales#89, number_sales#90]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt
index 1a9a4b4952efa..1890fdd496a94 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt
@@ -484,7 +484,7 @@ Join condition: None
(86) TakeOrderedAndProject
Input [12]: [channel#58, i_brand_id#44, i_class_id#45, i_category_id#46, sales#59, number_sales#60, channel#82, i_brand_id#69, i_class_id#70, i_category_id#71, sales#83, number_sales#84]
-Arguments: 100, 0, [i_brand_id#44 ASC NULLS FIRST, i_class_id#45 ASC NULLS FIRST, i_category_id#46 ASC NULLS FIRST], [channel#58, i_brand_id#44, i_class_id#45, i_category_id#46, sales#59, number_sales#60, channel#82, i_brand_id#69, i_class_id#70, i_category_id#71, sales#83, number_sales#84]
+Arguments: 100, [i_brand_id#44 ASC NULLS FIRST, i_class_id#45 ASC NULLS FIRST, i_category_id#46 ASC NULLS FIRST], [channel#58, i_brand_id#44, i_class_id#45, i_category_id#46, sales#59, number_sales#60, channel#82, i_brand_id#69, i_class_id#70, i_category_id#71, sales#83, number_sales#84]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.sf100/explain.txt
index 61a201b89f16f..55c39ed2e3288 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.sf100/explain.txt
@@ -154,7 +154,7 @@ Results [2]: [ca_zip#12, MakeDecimal(sum(UnscaledValue(cs_sales_price#2))#18,17,
(28) TakeOrderedAndProject
Input [2]: [ca_zip#12, sum(cs_sales_price)#19]
-Arguments: 100, 0, [ca_zip#12 ASC NULLS FIRST], [ca_zip#12, sum(cs_sales_price)#19]
+Arguments: 100, [ca_zip#12 ASC NULLS FIRST], [ca_zip#12, sum(cs_sales_price)#19]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/explain.txt
index 4f45cbc96bc08..fabc11130f172 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/explain.txt
@@ -124,7 +124,7 @@ Results [2]: [ca_zip#10, MakeDecimal(sum(UnscaledValue(cs_sales_price#2))#16,17,
(22) TakeOrderedAndProject
Input [2]: [ca_zip#10, sum(cs_sales_price)#17]
-Arguments: 100, 0, [ca_zip#10 ASC NULLS FIRST], [ca_zip#10, sum(cs_sales_price)#17]
+Arguments: 100, [ca_zip#10 ASC NULLS FIRST], [ca_zip#10, sum(cs_sales_price)#17]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/explain.txt
index db3a89f04672e..d61798f6ad06e 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/explain.txt
@@ -267,7 +267,7 @@ Results [15]: [i_item_id#14, i_item_desc#15, s_state#10, count(ss_quantity#5)#70
(49) TakeOrderedAndProject
Input [15]: [i_item_id#14, i_item_desc#15, s_state#10, store_sales_quantitycount#79, store_sales_quantityave#80, store_sales_quantitystdev#81, store_sales_quantitycov#82, as_store_returns_quantitycount#83, as_store_returns_quantityave#84, as_store_returns_quantitystdev#85, store_returns_quantitycov#86, catalog_sales_quantitycount#87, catalog_sales_quantityave#88, catalog_sales_quantitystdev#89, catalog_sales_quantitycov#90]
-Arguments: 100, 0, [i_item_id#14 ASC NULLS FIRST, i_item_desc#15 ASC NULLS FIRST, s_state#10 ASC NULLS FIRST], [i_item_id#14, i_item_desc#15, s_state#10, store_sales_quantitycount#79, store_sales_quantityave#80, store_sales_quantitystdev#81, store_sales_quantitycov#82, as_store_returns_quantitycount#83, as_store_returns_quantityave#84, as_store_returns_quantitystdev#85, store_returns_quantitycov#86, catalog_sales_quantitycount#87, catalog_sales_quantityave#88, catalog_sales_quantitystdev#89, catalog_sales_quantitycov#90]
+Arguments: 100, [i_item_id#14 ASC NULLS FIRST, i_item_desc#15 ASC NULLS FIRST, s_state#10 ASC NULLS FIRST], [i_item_id#14, i_item_desc#15, s_state#10, store_sales_quantitycount#79, store_sales_quantityave#80, store_sales_quantitystdev#81, store_sales_quantitycov#82, as_store_returns_quantitycount#83, as_store_returns_quantityave#84, as_store_returns_quantitystdev#85, store_returns_quantitycov#86, catalog_sales_quantitycount#87, catalog_sales_quantityave#88, catalog_sales_quantitystdev#89, catalog_sales_quantitycov#90]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/explain.txt
index abafb13f5cb71..d63f34438d5da 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/explain.txt
@@ -222,7 +222,7 @@ Results [15]: [i_item_id#27, i_item_desc#28, s_state#24, count(ss_quantity#5)#67
(40) TakeOrderedAndProject
Input [15]: [i_item_id#27, i_item_desc#28, s_state#24, store_sales_quantitycount#76, store_sales_quantityave#77, store_sales_quantitystdev#78, store_sales_quantitycov#79, as_store_returns_quantitycount#80, as_store_returns_quantityave#81, as_store_returns_quantitystdev#82, store_returns_quantitycov#83, catalog_sales_quantitycount#84, catalog_sales_quantityave#85, catalog_sales_quantitystdev#86, catalog_sales_quantitycov#87]
-Arguments: 100, 0, [i_item_id#27 ASC NULLS FIRST, i_item_desc#28 ASC NULLS FIRST, s_state#24 ASC NULLS FIRST], [i_item_id#27, i_item_desc#28, s_state#24, store_sales_quantitycount#76, store_sales_quantityave#77, store_sales_quantitystdev#78, store_sales_quantitycov#79, as_store_returns_quantitycount#80, as_store_returns_quantityave#81, as_store_returns_quantitystdev#82, store_returns_quantitycov#83, catalog_sales_quantitycount#84, catalog_sales_quantityave#85, catalog_sales_quantitystdev#86, catalog_sales_quantitycov#87]
+Arguments: 100, [i_item_id#27 ASC NULLS FIRST, i_item_desc#28 ASC NULLS FIRST, s_state#24 ASC NULLS FIRST], [i_item_id#27, i_item_desc#28, s_state#24, store_sales_quantitycount#76, store_sales_quantityave#77, store_sales_quantitystdev#78, store_sales_quantitycov#79, as_store_returns_quantitycount#80, as_store_returns_quantityave#81, as_store_returns_quantitystdev#82, store_returns_quantitycov#83, catalog_sales_quantitycount#84, catalog_sales_quantityave#85, catalog_sales_quantitystdev#86, catalog_sales_quantitycov#87]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.sf100/explain.txt
index a0c7ad80898a7..242ef26397ae2 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.sf100/explain.txt
@@ -268,7 +268,7 @@ Results [11]: [i_item_id#35, ca_country#36, ca_state#37, ca_county#38, avg(cast(
(49) TakeOrderedAndProject
Input [11]: [i_item_id#35, ca_country#36, ca_state#37, ca_county#38, agg1#76, agg2#77, agg3#78, agg4#79, agg5#80, agg6#81, agg7#82]
-Arguments: 100, 0, [ca_country#36 ASC NULLS FIRST, ca_state#37 ASC NULLS FIRST, ca_county#38 ASC NULLS FIRST, i_item_id#35 ASC NULLS FIRST], [i_item_id#35, ca_country#36, ca_state#37, ca_county#38, agg1#76, agg2#77, agg3#78, agg4#79, agg5#80, agg6#81, agg7#82]
+Arguments: 100, [ca_country#36 ASC NULLS FIRST, ca_state#37 ASC NULLS FIRST, ca_county#38 ASC NULLS FIRST, i_item_id#35 ASC NULLS FIRST], [i_item_id#35, ca_country#36, ca_state#37, ca_county#38, agg1#76, agg2#77, agg3#78, agg4#79, agg5#80, agg6#81, agg7#82]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/explain.txt
index 87acb1b8f31a2..d1b5044290a25 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/explain.txt
@@ -238,7 +238,7 @@ Results [11]: [i_item_id#33, ca_country#34, ca_state#35, ca_county#36, avg(cast(
(43) TakeOrderedAndProject
Input [11]: [i_item_id#33, ca_country#34, ca_state#35, ca_county#36, agg1#74, agg2#75, agg3#76, agg4#77, agg5#78, agg6#79, agg7#80]
-Arguments: 100, 0, [ca_country#34 ASC NULLS FIRST, ca_state#35 ASC NULLS FIRST, ca_county#36 ASC NULLS FIRST, i_item_id#33 ASC NULLS FIRST], [i_item_id#33, ca_country#34, ca_state#35, ca_county#36, agg1#74, agg2#75, agg3#76, agg4#77, agg5#78, agg6#79, agg7#80]
+Arguments: 100, [ca_country#34 ASC NULLS FIRST, ca_state#35 ASC NULLS FIRST, ca_county#36 ASC NULLS FIRST, i_item_id#33 ASC NULLS FIRST], [i_item_id#33, ca_country#34, ca_state#35, ca_county#36, agg1#74, agg2#75, agg3#76, agg4#77, agg5#78, agg6#79, agg7#80]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19.sf100/explain.txt
index 1c6127b9c236b..980ca7ba560c9 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19.sf100/explain.txt
@@ -225,7 +225,7 @@ Results [5]: [i_brand_id#8 AS brand_id#30, i_brand#9 AS brand#31, i_manufact_id#
(41) TakeOrderedAndProject
Input [5]: [brand_id#30, brand#31, i_manufact_id#10, i_manufact#11, ext_price#32]
-Arguments: 100, 0, [ext_price#32 DESC NULLS LAST, brand#31 ASC NULLS FIRST, brand_id#30 ASC NULLS FIRST, i_manufact_id#10 ASC NULLS FIRST, i_manufact#11 ASC NULLS FIRST], [brand_id#30, brand#31, i_manufact_id#10, i_manufact#11, ext_price#32]
+Arguments: 100, [ext_price#32 DESC NULLS LAST, brand#31 ASC NULLS FIRST, brand_id#30 ASC NULLS FIRST, i_manufact_id#10 ASC NULLS FIRST, i_manufact#11 ASC NULLS FIRST], [brand_id#30, brand#31, i_manufact_id#10, i_manufact#11, ext_price#32]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19/explain.txt
index 8c2fb5402dfe3..e321ddc156860 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19/explain.txt
@@ -218,5 +218,5 @@ Results [5]: [i_brand_id#11 AS brand_id#30, i_brand#12 AS brand#31, i_manufact_i
(39) TakeOrderedAndProject
Input [5]: [brand_id#30, brand#31, i_manufact_id#13, i_manufact#14, ext_price#32]
-Arguments: 100, 0, [ext_price#32 DESC NULLS LAST, brand#31 ASC NULLS FIRST, brand_id#30 ASC NULLS FIRST, i_manufact_id#13 ASC NULLS FIRST, i_manufact#14 ASC NULLS FIRST], [brand_id#30, brand#31, i_manufact_id#13, i_manufact#14, ext_price#32]
+Arguments: 100, [ext_price#32 DESC NULLS LAST, brand#31 ASC NULLS FIRST, brand_id#30 ASC NULLS FIRST, i_manufact_id#13 ASC NULLS FIRST, i_manufact#14 ASC NULLS FIRST], [brand_id#30, brand#31, i_manufact_id#13, i_manufact#14, ext_price#32]
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.sf100/explain.txt
index 6f86939e32f8b..fb32206c5d32d 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.sf100/explain.txt
@@ -126,7 +126,7 @@ Input [9]: [i_item_desc#8, i_category#11, i_class#10, i_current_price#9, itemrev
(23) TakeOrderedAndProject
Input [7]: [i_item_desc#8, i_category#11, i_class#10, i_current_price#9, itemrevenue#18, revenueratio#23, i_item_id#7]
-Arguments: 100, 0, [i_category#11 ASC NULLS FIRST, i_class#10 ASC NULLS FIRST, i_item_id#7 ASC NULLS FIRST, i_item_desc#8 ASC NULLS FIRST, revenueratio#23 ASC NULLS FIRST], [i_item_desc#8, i_category#11, i_class#10, i_current_price#9, itemrevenue#18, revenueratio#23]
+Arguments: 100, [i_category#11 ASC NULLS FIRST, i_class#10 ASC NULLS FIRST, i_item_id#7 ASC NULLS FIRST, i_item_desc#8 ASC NULLS FIRST, revenueratio#23 ASC NULLS FIRST], [i_item_desc#8, i_category#11, i_class#10, i_current_price#9, itemrevenue#18, revenueratio#23]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/explain.txt
index 3c8daf552647d..5fb56166d8cef 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/explain.txt
@@ -111,7 +111,7 @@ Input [9]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemreve
(20) TakeOrderedAndProject
Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#17, revenueratio#22, i_item_id#6]
-Arguments: 100, 0, [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#22 ASC NULLS FIRST], [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#17, revenueratio#22]
+Arguments: 100, [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#22 ASC NULLS FIRST], [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#17, revenueratio#22]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.sf100/explain.txt
index df8420493d7dc..96e1bacaa55ae 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.sf100/explain.txt
@@ -134,7 +134,7 @@ Condition : ((isnotnull(inv_before#22) AND isnotnull(inv_after#23)) AND (((inv_b
(24) TakeOrderedAndProject
Input [4]: [w_warehouse_name#13, i_item_id#7, inv_before#22, inv_after#23]
-Arguments: 100, 0, [w_warehouse_name#13 ASC NULLS FIRST, i_item_id#7 ASC NULLS FIRST], [w_warehouse_name#13, i_item_id#7, inv_before#22, inv_after#23]
+Arguments: 100, [w_warehouse_name#13 ASC NULLS FIRST, i_item_id#7 ASC NULLS FIRST], [w_warehouse_name#13, i_item_id#7, inv_before#22, inv_after#23]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/explain.txt
index 318b7dd31666f..01bba5ee5f966 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/explain.txt
@@ -134,7 +134,7 @@ Condition : ((isnotnull(inv_before#22) AND isnotnull(inv_after#23)) AND (((inv_b
(24) TakeOrderedAndProject
Input [4]: [w_warehouse_name#7, i_item_id#10, inv_before#22, inv_after#23]
-Arguments: 100, 0, [w_warehouse_name#7 ASC NULLS FIRST, i_item_id#10 ASC NULLS FIRST], [w_warehouse_name#7, i_item_id#10, inv_before#22, inv_after#23]
+Arguments: 100, [w_warehouse_name#7 ASC NULLS FIRST, i_item_id#10 ASC NULLS FIRST], [w_warehouse_name#7, i_item_id#10, inv_before#22, inv_after#23]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22.sf100/explain.txt
index 6a7908f51c9fb..4258a9c1a71c1 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22.sf100/explain.txt
@@ -144,7 +144,7 @@ Results [5]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, avg(inv_
(26) TakeOrderedAndProject
Input [5]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, qoh#27]
-Arguments: 100, 0, [qoh#27 ASC NULLS FIRST, i_product_name#16 ASC NULLS FIRST, i_brand#17 ASC NULLS FIRST, i_class#18 ASC NULLS FIRST, i_category#19 ASC NULLS FIRST], [i_product_name#16, i_brand#17, i_class#18, i_category#19, qoh#27]
+Arguments: 100, [qoh#27 ASC NULLS FIRST, i_product_name#16 ASC NULLS FIRST, i_brand#17 ASC NULLS FIRST, i_class#18 ASC NULLS FIRST, i_category#19 ASC NULLS FIRST], [i_product_name#16, i_brand#17, i_class#18, i_category#19, qoh#27]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22/explain.txt
index 5bc98ac492e8b..c1b0b42178114 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22/explain.txt
@@ -129,7 +129,7 @@ Results [5]: [i_product_name#15, i_brand#16, i_class#17, i_category#18, avg(inv_
(23) TakeOrderedAndProject
Input [5]: [i_product_name#15, i_brand#16, i_class#17, i_category#18, qoh#26]
-Arguments: 100, 0, [qoh#26 ASC NULLS FIRST, i_product_name#15 ASC NULLS FIRST, i_brand#16 ASC NULLS FIRST, i_class#17 ASC NULLS FIRST, i_category#18 ASC NULLS FIRST], [i_product_name#15, i_brand#16, i_class#17, i_category#18, qoh#26]
+Arguments: 100, [qoh#26 ASC NULLS FIRST, i_product_name#15 ASC NULLS FIRST, i_brand#16 ASC NULLS FIRST, i_class#17 ASC NULLS FIRST, i_category#18 ASC NULLS FIRST], [i_product_name#15, i_brand#16, i_class#17, i_category#18, qoh#26]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.sf100/explain.txt
index cb2a4025bf913..d90708aa6d352 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.sf100/explain.txt
@@ -712,7 +712,7 @@ Results [3]: [c_last_name#62, c_first_name#61, sum((cast(ws_quantity#53 as decim
(129) TakeOrderedAndProject
Input [3]: [c_last_name#42, c_first_name#41, sales#50]
-Arguments: 100, 0, [c_last_name#42 ASC NULLS FIRST, c_first_name#41 ASC NULLS FIRST, sales#50 ASC NULLS FIRST], [c_last_name#42, c_first_name#41, sales#50]
+Arguments: 100, [c_last_name#42 ASC NULLS FIRST, c_first_name#41 ASC NULLS FIRST, sales#50 ASC NULLS FIRST], [c_last_name#42, c_first_name#41, sales#50]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/explain.txt
index 671568ae914db..d723aa3a322e3 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/explain.txt
@@ -484,7 +484,7 @@ Results [3]: [c_last_name#60, c_first_name#59, sum((cast(ws_quantity#54 as decim
(87) TakeOrderedAndProject
Input [3]: [c_last_name#41, c_first_name#40, sales#51]
-Arguments: 100, 0, [c_last_name#41 ASC NULLS FIRST, c_first_name#40 ASC NULLS FIRST, sales#51 ASC NULLS FIRST], [c_last_name#41, c_first_name#40, sales#51]
+Arguments: 100, [c_last_name#41 ASC NULLS FIRST, c_first_name#40 ASC NULLS FIRST, sales#51 ASC NULLS FIRST], [c_last_name#41, c_first_name#40, sales#51]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.sf100/explain.txt
index 01661abebc10a..fc55789fab16a 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.sf100/explain.txt
@@ -267,7 +267,7 @@ Results [7]: [i_item_id#15, i_item_desc#16, s_store_id#10, s_store_name#11, Make
(49) TakeOrderedAndProject
Input [7]: [i_item_id#15, i_item_desc#16, s_store_id#10, s_store_name#11, store_sales_profit#44, store_returns_loss#45, catalog_sales_profit#46]
-Arguments: 100, 0, [i_item_id#15 ASC NULLS FIRST, i_item_desc#16 ASC NULLS FIRST, s_store_id#10 ASC NULLS FIRST, s_store_name#11 ASC NULLS FIRST], [i_item_id#15, i_item_desc#16, s_store_id#10, s_store_name#11, store_sales_profit#44, store_returns_loss#45, catalog_sales_profit#46]
+Arguments: 100, [i_item_id#15 ASC NULLS FIRST, i_item_desc#16 ASC NULLS FIRST, s_store_id#10 ASC NULLS FIRST, s_store_name#11 ASC NULLS FIRST], [i_item_id#15, i_item_desc#16, s_store_id#10, s_store_name#11, store_sales_profit#44, store_returns_loss#45, catalog_sales_profit#46]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/explain.txt
index 51034f5a10e8a..b9d08511b6dce 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/explain.txt
@@ -222,7 +222,7 @@ Results [7]: [i_item_id#28, i_item_desc#29, s_store_id#24, s_store_name#25, Make
(40) TakeOrderedAndProject
Input [7]: [i_item_id#28, i_item_desc#29, s_store_id#24, s_store_name#25, store_sales_profit#41, store_returns_loss#42, catalog_sales_profit#43]
-Arguments: 100, 0, [i_item_id#28 ASC NULLS FIRST, i_item_desc#29 ASC NULLS FIRST, s_store_id#24 ASC NULLS FIRST, s_store_name#25 ASC NULLS FIRST], [i_item_id#28, i_item_desc#29, s_store_id#24, s_store_name#25, store_sales_profit#41, store_returns_loss#42, catalog_sales_profit#43]
+Arguments: 100, [i_item_id#28 ASC NULLS FIRST, i_item_desc#29 ASC NULLS FIRST, s_store_id#24 ASC NULLS FIRST, s_store_name#25 ASC NULLS FIRST], [i_item_id#28, i_item_desc#29, s_store_id#24, s_store_name#25, store_sales_profit#41, store_returns_loss#42, catalog_sales_profit#43]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.sf100/explain.txt
index 7ab0a5de79d73..2a9a6d00ba3c8 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.sf100/explain.txt
@@ -167,7 +167,7 @@ Results [5]: [i_item_id#21, avg(cs_quantity#4)#40 AS agg1#44, cast((avg(Unscaled
(30) TakeOrderedAndProject
Input [5]: [i_item_id#21, agg1#44, agg2#45, agg3#46, agg4#47]
-Arguments: 100, 0, [i_item_id#21 ASC NULLS FIRST], [i_item_id#21, agg1#44, agg2#45, agg3#46, agg4#47]
+Arguments: 100, [i_item_id#21 ASC NULLS FIRST], [i_item_id#21, agg1#44, agg2#45, agg3#46, agg4#47]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/explain.txt
index c4b3f35bf514e..bde65bfda5cbf 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/explain.txt
@@ -167,7 +167,7 @@ Results [5]: [i_item_id#17, avg(cs_quantity#4)#40 AS agg1#44, cast((avg(Unscaled
(30) TakeOrderedAndProject
Input [5]: [i_item_id#17, agg1#44, agg2#45, agg3#46, agg4#47]
-Arguments: 100, 0, [i_item_id#17 ASC NULLS FIRST], [i_item_id#17, agg1#44, agg2#45, agg3#46, agg4#47]
+Arguments: 100, [i_item_id#17 ASC NULLS FIRST], [i_item_id#17, agg1#44, agg2#45, agg3#46, agg4#47]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.sf100/explain.txt
index 8adc29aba28e4..7600710283f2a 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.sf100/explain.txt
@@ -167,7 +167,7 @@ Results [7]: [i_item_id#22, s_state#23, cast((shiftright(spark_grouping_id#24, 0
(30) TakeOrderedAndProject
Input [7]: [i_item_id#22, s_state#23, g_state#46, agg1#47, agg2#48, agg3#49, agg4#50]
-Arguments: 100, 0, [i_item_id#22 ASC NULLS FIRST, s_state#23 ASC NULLS FIRST], [i_item_id#22, s_state#23, g_state#46, agg1#47, agg2#48, agg3#49, agg4#50]
+Arguments: 100, [i_item_id#22 ASC NULLS FIRST, s_state#23 ASC NULLS FIRST], [i_item_id#22, s_state#23, g_state#46, agg1#47, agg2#48, agg3#49, agg4#50]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/explain.txt
index 8adc29aba28e4..7600710283f2a 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/explain.txt
@@ -167,7 +167,7 @@ Results [7]: [i_item_id#22, s_state#23, cast((shiftright(spark_grouping_id#24, 0
(30) TakeOrderedAndProject
Input [7]: [i_item_id#22, s_state#23, g_state#46, agg1#47, agg2#48, agg3#49, agg4#50]
-Arguments: 100, 0, [i_item_id#22 ASC NULLS FIRST, s_state#23 ASC NULLS FIRST], [i_item_id#22, s_state#23, g_state#46, agg1#47, agg2#48, agg3#49, agg4#50]
+Arguments: 100, [i_item_id#22 ASC NULLS FIRST, s_state#23 ASC NULLS FIRST], [i_item_id#22, s_state#23, g_state#46, agg1#47, agg2#48, agg3#49, agg4#50]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.sf100/explain.txt
index 12e850cf82ce8..221439075d24d 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.sf100/explain.txt
@@ -267,7 +267,7 @@ Results [7]: [i_item_id#15, i_item_desc#16, s_store_id#10, s_store_name#11, sum(
(49) TakeOrderedAndProject
Input [7]: [i_item_id#15, i_item_desc#16, s_store_id#10, s_store_name#11, store_sales_quantity#45, store_returns_quantity#46, catalog_sales_quantity#47]
-Arguments: 100, 0, [i_item_id#15 ASC NULLS FIRST, i_item_desc#16 ASC NULLS FIRST, s_store_id#10 ASC NULLS FIRST, s_store_name#11 ASC NULLS FIRST], [i_item_id#15, i_item_desc#16, s_store_id#10, s_store_name#11, store_sales_quantity#45, store_returns_quantity#46, catalog_sales_quantity#47]
+Arguments: 100, [i_item_id#15 ASC NULLS FIRST, i_item_desc#16 ASC NULLS FIRST, s_store_id#10 ASC NULLS FIRST, s_store_name#11 ASC NULLS FIRST], [i_item_id#15, i_item_desc#16, s_store_id#10, s_store_name#11, store_sales_quantity#45, store_returns_quantity#46, catalog_sales_quantity#47]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29/explain.txt
index 3bc469d6511ee..86096ab89650c 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29/explain.txt
@@ -222,7 +222,7 @@ Results [7]: [i_item_id#29, i_item_desc#30, s_store_id#25, s_store_name#26, sum(
(40) TakeOrderedAndProject
Input [7]: [i_item_id#29, i_item_desc#30, s_store_id#25, s_store_name#26, store_sales_quantity#42, store_returns_quantity#43, catalog_sales_quantity#44]
-Arguments: 100, 0, [i_item_id#29 ASC NULLS FIRST, i_item_desc#30 ASC NULLS FIRST, s_store_id#25 ASC NULLS FIRST, s_store_name#26 ASC NULLS FIRST], [i_item_id#29, i_item_desc#30, s_store_id#25, s_store_name#26, store_sales_quantity#42, store_returns_quantity#43, catalog_sales_quantity#44]
+Arguments: 100, [i_item_id#29 ASC NULLS FIRST, i_item_desc#30 ASC NULLS FIRST, s_store_id#25 ASC NULLS FIRST, s_store_name#26 ASC NULLS FIRST], [i_item_id#29, i_item_desc#30, s_store_id#25, s_store_name#26, store_sales_quantity#42, store_returns_quantity#43, catalog_sales_quantity#44]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3.sf100/explain.txt
index 4796d84bd0540..8ec42f473ee9e 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3.sf100/explain.txt
@@ -96,7 +96,7 @@ Results [4]: [d_year#11, i_brand_id#6 AS brand_id#16, i_brand#7 AS brand#17, Mak
(17) TakeOrderedAndProject
Input [4]: [d_year#11, brand_id#16, brand#17, sum_agg#18]
-Arguments: 100, 0, [d_year#11 ASC NULLS FIRST, sum_agg#18 DESC NULLS LAST, brand_id#16 ASC NULLS FIRST], [d_year#11, brand_id#16, brand#17, sum_agg#18]
+Arguments: 100, [d_year#11 ASC NULLS FIRST, sum_agg#18 DESC NULLS LAST, brand_id#16 ASC NULLS FIRST], [d_year#11, brand_id#16, brand#17, sum_agg#18]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3/explain.txt
index b9a6526961f14..77eb7fefd4d05 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3/explain.txt
@@ -119,5 +119,5 @@ Results [4]: [d_year#2, i_brand_id#9 AS brand_id#17, i_brand#10 AS brand#18, Mak
(21) TakeOrderedAndProject
Input [4]: [d_year#2, brand_id#17, brand#18, sum_agg#19]
-Arguments: 100, 0, [d_year#2 ASC NULLS FIRST, sum_agg#19 DESC NULLS LAST, brand_id#17 ASC NULLS FIRST], [d_year#2, brand_id#17, brand#18, sum_agg#19]
+Arguments: 100, [d_year#2 ASC NULLS FIRST, sum_agg#19 DESC NULLS LAST, brand_id#17 ASC NULLS FIRST], [d_year#2, brand_id#17, brand#18, sum_agg#19]
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.sf100/explain.txt
index 2ba30392ddf01..7183a1224830c 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.sf100/explain.txt
@@ -308,7 +308,7 @@ Input [16]: [c_customer_id#2, c_salutation#4, c_first_name#5, c_last_name#6, c_p
(55) TakeOrderedAndProject
Input [13]: [c_customer_id#2, c_salutation#4, c_first_name#5, c_last_name#6, c_preferred_cust_flag#7, c_birth_day#8, c_birth_month#9, c_birth_year#10, c_birth_country#11, c_login#12, c_email_address#13, c_last_review_date#14, ctr_total_return#35]
-Arguments: 100, 0, [c_customer_id#2 ASC NULLS FIRST, c_salutation#4 ASC NULLS FIRST, c_first_name#5 ASC NULLS FIRST, c_last_name#6 ASC NULLS FIRST, c_preferred_cust_flag#7 ASC NULLS FIRST, c_birth_day#8 ASC NULLS FIRST, c_birth_month#9 ASC NULLS FIRST, c_birth_year#10 ASC NULLS FIRST, c_birth_country#11 ASC NULLS FIRST, c_login#12 ASC NULLS FIRST, c_email_address#13 ASC NULLS FIRST, c_last_review_date#14 ASC NULLS FIRST, ctr_total_return#35 ASC NULLS FIRST], [c_customer_id#2, c_salutation#4, c_first_name#5, c_last_name#6, c_preferred_cust_flag#7, c_birth_day#8, c_birth_month#9, c_birth_year#10, c_birth_country#11, c_login#12, c_email_address#13, c_last_review_date#14, ctr_total_return#35]
+Arguments: 100, [c_customer_id#2 ASC NULLS FIRST, c_salutation#4 ASC NULLS FIRST, c_first_name#5 ASC NULLS FIRST, c_last_name#6 ASC NULLS FIRST, c_preferred_cust_flag#7 ASC NULLS FIRST, c_birth_day#8 ASC NULLS FIRST, c_birth_month#9 ASC NULLS FIRST, c_birth_year#10 ASC NULLS FIRST, c_birth_country#11 ASC NULLS FIRST, c_login#12 ASC NULLS FIRST, c_email_address#13 ASC NULLS FIRST, c_last_review_date#14 ASC NULLS FIRST, ctr_total_return#35 ASC NULLS FIRST], [c_customer_id#2, c_salutation#4, c_first_name#5, c_last_name#6, c_preferred_cust_flag#7, c_birth_day#8, c_birth_month#9, c_birth_year#10, c_birth_country#11, c_login#12, c_email_address#13, c_last_review_date#14, ctr_total_return#35]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/explain.txt
index 8e3c98d415941..e6f2d600efd4c 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/explain.txt
@@ -278,7 +278,7 @@ Input [15]: [ctr_total_return#16, c_customer_id#30, c_current_addr_sk#31, c_salu
(49) TakeOrderedAndProject
Input [13]: [c_customer_id#30, c_salutation#32, c_first_name#33, c_last_name#34, c_preferred_cust_flag#35, c_birth_day#36, c_birth_month#37, c_birth_year#38, c_birth_country#39, c_login#40, c_email_address#41, c_last_review_date#42, ctr_total_return#16]
-Arguments: 100, 0, [c_customer_id#30 ASC NULLS FIRST, c_salutation#32 ASC NULLS FIRST, c_first_name#33 ASC NULLS FIRST, c_last_name#34 ASC NULLS FIRST, c_preferred_cust_flag#35 ASC NULLS FIRST, c_birth_day#36 ASC NULLS FIRST, c_birth_month#37 ASC NULLS FIRST, c_birth_year#38 ASC NULLS FIRST, c_birth_country#39 ASC NULLS FIRST, c_login#40 ASC NULLS FIRST, c_email_address#41 ASC NULLS FIRST, c_last_review_date#42 ASC NULLS FIRST, ctr_total_return#16 ASC NULLS FIRST], [c_customer_id#30, c_salutation#32, c_first_name#33, c_last_name#34, c_preferred_cust_flag#35, c_birth_day#36, c_birth_month#37, c_birth_year#38, c_birth_country#39, c_login#40, c_email_address#41, c_last_review_date#42, ctr_total_return#16]
+Arguments: 100, [c_customer_id#30 ASC NULLS FIRST, c_salutation#32 ASC NULLS FIRST, c_first_name#33 ASC NULLS FIRST, c_last_name#34 ASC NULLS FIRST, c_preferred_cust_flag#35 ASC NULLS FIRST, c_birth_day#36 ASC NULLS FIRST, c_birth_month#37 ASC NULLS FIRST, c_birth_year#38 ASC NULLS FIRST, c_birth_country#39 ASC NULLS FIRST, c_login#40 ASC NULLS FIRST, c_email_address#41 ASC NULLS FIRST, c_last_review_date#42 ASC NULLS FIRST, ctr_total_return#16 ASC NULLS FIRST], [c_customer_id#30, c_salutation#32, c_first_name#33, c_last_name#34, c_preferred_cust_flag#35, c_birth_day#36, c_birth_month#37, c_birth_year#38, c_birth_country#39, c_login#40, c_email_address#41, c_last_review_date#42, ctr_total_return#16]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.sf100/explain.txt
index ea53cb5843b84..06bcfd38eae48 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.sf100/explain.txt
@@ -150,7 +150,7 @@ Input [2]: [cs_sold_date_sk#19, d_date_sk#20]
(27) CollectLimit
Input [1]: [excess discount amount #21]
-Arguments: 100, 0
+Arguments: 100
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/explain.txt
index f9e2ee52e005b..c00a83d6bd491 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/explain.txt
@@ -150,7 +150,7 @@ Input [2]: [cs_sold_date_sk#3, d_date_sk#20]
(27) CollectLimit
Input [1]: [excess discount amount #21]
-Arguments: 100, 0
+Arguments: 100
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/explain.txt
index 6b14fd1b3824d..b1e601e2093ae 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/explain.txt
@@ -354,7 +354,7 @@ Results [2]: [i_manufact_id#8, sum(total_sales#20)#52 AS total_sales#53]
(63) TakeOrderedAndProject
Input [2]: [i_manufact_id#8, total_sales#53]
-Arguments: 100, 0, [total_sales#53 ASC NULLS FIRST], [i_manufact_id#8, total_sales#53]
+Arguments: 100, [total_sales#53 ASC NULLS FIRST], [i_manufact_id#8, total_sales#53]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt
index b791152cb058c..2dc29dcb3a78e 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt
@@ -354,7 +354,7 @@ Results [2]: [i_manufact_id#11, sum(total_sales#20)#52 AS total_sales#53]
(63) TakeOrderedAndProject
Input [2]: [i_manufact_id#11, total_sales#53]
-Arguments: 100, 0, [total_sales#53 ASC NULLS FIRST], [i_manufact_id#11, total_sales#53]
+Arguments: 100, [total_sales#53 ASC NULLS FIRST], [i_manufact_id#11, total_sales#53]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.sf100/explain.txt
index eaa48e4451ba1..1a3d3ccb75e06 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.sf100/explain.txt
@@ -287,7 +287,7 @@ Results [18]: [ca_state#22, cd_gender#26, cd_marital_status#27, count(1)#59 AS c
(53) TakeOrderedAndProject
Input [18]: [ca_state#22, cd_gender#26, cd_marital_status#27, cnt1#69, min(cd_dep_count)#70, max(cd_dep_count)#71, avg(cd_dep_count)#72, cd_dep_employed_count#29, cnt2#73, min(cd_dep_employed_count)#74, max(cd_dep_employed_count)#75, avg(cd_dep_employed_count)#76, cd_dep_college_count#30, cnt3#77, min(cd_dep_college_count)#78, max(cd_dep_college_count)#79, avg(cd_dep_college_count)#80, cd_dep_count#28]
-Arguments: 100, 0, [ca_state#22 ASC NULLS FIRST, cd_gender#26 ASC NULLS FIRST, cd_marital_status#27 ASC NULLS FIRST, cd_dep_count#28 ASC NULLS FIRST, cd_dep_employed_count#29 ASC NULLS FIRST, cd_dep_college_count#30 ASC NULLS FIRST], [ca_state#22, cd_gender#26, cd_marital_status#27, cnt1#69, min(cd_dep_count)#70, max(cd_dep_count)#71, avg(cd_dep_count)#72, cd_dep_employed_count#29, cnt2#73, min(cd_dep_employed_count)#74, max(cd_dep_employed_count)#75, avg(cd_dep_employed_count)#76, cd_dep_college_count#30, cnt3#77, min(cd_dep_college_count)#78, max(cd_dep_college_count)#79, avg(cd_dep_college_count)#80]
+Arguments: 100, [ca_state#22 ASC NULLS FIRST, cd_gender#26 ASC NULLS FIRST, cd_marital_status#27 ASC NULLS FIRST, cd_dep_count#28 ASC NULLS FIRST, cd_dep_employed_count#29 ASC NULLS FIRST, cd_dep_college_count#30 ASC NULLS FIRST], [ca_state#22, cd_gender#26, cd_marital_status#27, cnt1#69, min(cd_dep_count)#70, max(cd_dep_count)#71, avg(cd_dep_count)#72, cd_dep_employed_count#29, cnt2#73, min(cd_dep_employed_count)#74, max(cd_dep_employed_count)#75, avg(cd_dep_employed_count)#76, cd_dep_college_count#30, cnt3#77, min(cd_dep_college_count)#78, max(cd_dep_college_count)#79, avg(cd_dep_college_count)#80]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/explain.txt
index 17fcde4197573..e5a9030386262 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/explain.txt
@@ -232,7 +232,7 @@ Results [18]: [ca_state#20, cd_gender#23, cd_marital_status#24, count(1)#56 AS c
(42) TakeOrderedAndProject
Input [18]: [ca_state#20, cd_gender#23, cd_marital_status#24, cnt1#66, min(cd_dep_count)#67, max(cd_dep_count)#68, avg(cd_dep_count)#69, cd_dep_employed_count#26, cnt2#70, min(cd_dep_employed_count)#71, max(cd_dep_employed_count)#72, avg(cd_dep_employed_count)#73, cd_dep_college_count#27, cnt3#74, min(cd_dep_college_count)#75, max(cd_dep_college_count)#76, avg(cd_dep_college_count)#77, cd_dep_count#25]
-Arguments: 100, 0, [ca_state#20 ASC NULLS FIRST, cd_gender#23 ASC NULLS FIRST, cd_marital_status#24 ASC NULLS FIRST, cd_dep_count#25 ASC NULLS FIRST, cd_dep_employed_count#26 ASC NULLS FIRST, cd_dep_college_count#27 ASC NULLS FIRST], [ca_state#20, cd_gender#23, cd_marital_status#24, cnt1#66, min(cd_dep_count)#67, max(cd_dep_count)#68, avg(cd_dep_count)#69, cd_dep_employed_count#26, cnt2#70, min(cd_dep_employed_count)#71, max(cd_dep_employed_count)#72, avg(cd_dep_employed_count)#73, cd_dep_college_count#27, cnt3#74, min(cd_dep_college_count)#75, max(cd_dep_college_count)#76, avg(cd_dep_college_count)#77]
+Arguments: 100, [ca_state#20 ASC NULLS FIRST, cd_gender#23 ASC NULLS FIRST, cd_marital_status#24 ASC NULLS FIRST, cd_dep_count#25 ASC NULLS FIRST, cd_dep_employed_count#26 ASC NULLS FIRST, cd_dep_college_count#27 ASC NULLS FIRST], [ca_state#20, cd_gender#23, cd_marital_status#24, cnt1#66, min(cd_dep_count)#67, max(cd_dep_count)#68, avg(cd_dep_count)#69, cd_dep_employed_count#26, cnt2#70, min(cd_dep_employed_count)#71, max(cd_dep_employed_count)#72, avg(cd_dep_employed_count)#73, cd_dep_college_count#27, cnt3#74, min(cd_dep_college_count)#75, max(cd_dep_college_count)#76, avg(cd_dep_college_count)#77]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.sf100/explain.txt
index ee103cbe72375..4f4ba33f89cb1 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.sf100/explain.txt
@@ -154,7 +154,7 @@ Input [8]: [gross_margin#25, i_category#15, i_class#16, lochierarchy#26, _w1#27,
(28) TakeOrderedAndProject
Input [5]: [gross_margin#25, i_category#15, i_class#16, lochierarchy#26, rank_within_parent#31]
-Arguments: 100, 0, [lochierarchy#26 DESC NULLS LAST, CASE WHEN (lochierarchy#26 = 0) THEN i_category#15 END ASC NULLS FIRST, rank_within_parent#31 ASC NULLS FIRST], [gross_margin#25, i_category#15, i_class#16, lochierarchy#26, rank_within_parent#31]
+Arguments: 100, [lochierarchy#26 DESC NULLS LAST, CASE WHEN (lochierarchy#26 = 0) THEN i_category#15 END ASC NULLS FIRST, rank_within_parent#31 ASC NULLS FIRST], [gross_margin#25, i_category#15, i_class#16, lochierarchy#26, rank_within_parent#31]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/explain.txt
index bae0eb2a7a0ae..987b506904a13 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/explain.txt
@@ -154,7 +154,7 @@ Input [8]: [gross_margin#25, i_category#15, i_class#16, lochierarchy#26, _w1#27,
(28) TakeOrderedAndProject
Input [5]: [gross_margin#25, i_category#15, i_class#16, lochierarchy#26, rank_within_parent#31]
-Arguments: 100, 0, [lochierarchy#26 DESC NULLS LAST, CASE WHEN (lochierarchy#26 = 0) THEN i_category#15 END ASC NULLS FIRST, rank_within_parent#31 ASC NULLS FIRST], [gross_margin#25, i_category#15, i_class#16, lochierarchy#26, rank_within_parent#31]
+Arguments: 100, [lochierarchy#26 DESC NULLS LAST, CASE WHEN (lochierarchy#26 = 0) THEN i_category#15 END ASC NULLS FIRST, rank_within_parent#31 ASC NULLS FIRST], [gross_margin#25, i_category#15, i_class#16, lochierarchy#26, rank_within_parent#31]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.sf100/explain.txt
index 0ee0782e60580..df9567ae2c5e9 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.sf100/explain.txt
@@ -154,7 +154,7 @@ Results [3]: [i_item_id#2, i_item_desc#3, i_current_price#4]
(28) TakeOrderedAndProject
Input [3]: [i_item_id#2, i_item_desc#3, i_current_price#4]
-Arguments: 100, 0, [i_item_id#2 ASC NULLS FIRST], [i_item_id#2, i_item_desc#3, i_current_price#4]
+Arguments: 100, [i_item_id#2 ASC NULLS FIRST], [i_item_id#2, i_item_desc#3, i_current_price#4]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37/explain.txt
index cba29c535f245..956b8c98a30c2 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37/explain.txt
@@ -139,7 +139,7 @@ Results [3]: [i_item_id#2, i_item_desc#3, i_current_price#4]
(25) TakeOrderedAndProject
Input [3]: [i_item_id#2, i_item_desc#3, i_current_price#4]
-Arguments: 100, 0, [i_item_id#2 ASC NULLS FIRST], [i_item_id#2, i_item_desc#3, i_current_price#4]
+Arguments: 100, [i_item_id#2 ASC NULLS FIRST], [i_item_id#2, i_item_desc#3, i_current_price#4]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.sf100/explain.txt
index b89e47ac91a85..f65209eecd086 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.sf100/explain.txt
@@ -661,7 +661,7 @@ Input [13]: [customer_id#26, customer_id#52, customer_first_name#53, customer_la
(120) TakeOrderedAndProject
Input [7]: [customer_id#52, customer_first_name#53, customer_last_name#54, customer_preferred_cust_flag#55, customer_birth_country#56, customer_login#57, customer_email_address#58]
-Arguments: 100, 0, [customer_id#52 ASC NULLS FIRST, customer_first_name#53 ASC NULLS FIRST, customer_last_name#54 ASC NULLS FIRST, customer_preferred_cust_flag#55 ASC NULLS FIRST, customer_birth_country#56 ASC NULLS FIRST, customer_login#57 ASC NULLS FIRST, customer_email_address#58 ASC NULLS FIRST], [customer_id#52, customer_first_name#53, customer_last_name#54, customer_preferred_cust_flag#55, customer_birth_country#56, customer_login#57, customer_email_address#58]
+Arguments: 100, [customer_id#52 ASC NULLS FIRST, customer_first_name#53 ASC NULLS FIRST, customer_last_name#54 ASC NULLS FIRST, customer_preferred_cust_flag#55 ASC NULLS FIRST, customer_birth_country#56 ASC NULLS FIRST, customer_login#57 ASC NULLS FIRST, customer_email_address#58 ASC NULLS FIRST], [customer_id#52, customer_first_name#53, customer_last_name#54, customer_preferred_cust_flag#55, customer_birth_country#56, customer_login#57, customer_email_address#58]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/explain.txt
index 4db6fd4b778f8..a4fcf5deb9473 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/explain.txt
@@ -626,7 +626,7 @@ Input [13]: [customer_id#25, customer_id#50, customer_first_name#51, customer_la
(110) TakeOrderedAndProject
Input [7]: [customer_id#50, customer_first_name#51, customer_last_name#52, customer_preferred_cust_flag#53, customer_birth_country#54, customer_login#55, customer_email_address#56]
-Arguments: 100, 0, [customer_id#50 ASC NULLS FIRST, customer_first_name#51 ASC NULLS FIRST, customer_last_name#52 ASC NULLS FIRST, customer_preferred_cust_flag#53 ASC NULLS FIRST, customer_birth_country#54 ASC NULLS FIRST, customer_login#55 ASC NULLS FIRST, customer_email_address#56 ASC NULLS FIRST], [customer_id#50, customer_first_name#51, customer_last_name#52, customer_preferred_cust_flag#53, customer_birth_country#54, customer_login#55, customer_email_address#56]
+Arguments: 100, [customer_id#50 ASC NULLS FIRST, customer_first_name#51 ASC NULLS FIRST, customer_last_name#52 ASC NULLS FIRST, customer_preferred_cust_flag#53 ASC NULLS FIRST, customer_birth_country#54 ASC NULLS FIRST, customer_login#55 ASC NULLS FIRST, customer_email_address#56 ASC NULLS FIRST], [customer_id#50, customer_first_name#51, customer_last_name#52, customer_preferred_cust_flag#53, customer_birth_country#54, customer_login#55, customer_email_address#56]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40.sf100/explain.txt
index a4daf4bc2a6dd..f07851125f451 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40.sf100/explain.txt
@@ -182,7 +182,7 @@ Results [4]: [w_state#20, i_item_id#14, sum(CASE WHEN (d_date#18 < 2000-03-11) T
(33) TakeOrderedAndProject
Input [4]: [w_state#20, i_item_id#14, sales_before#33, sales_after#34]
-Arguments: 100, 0, [w_state#20 ASC NULLS FIRST, i_item_id#14 ASC NULLS FIRST], [w_state#20, i_item_id#14, sales_before#33, sales_after#34]
+Arguments: 100, [w_state#20 ASC NULLS FIRST, i_item_id#14 ASC NULLS FIRST], [w_state#20, i_item_id#14, sales_before#33, sales_after#34]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/explain.txt
index 5e9fbc075e59b..239a44282ea45 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/explain.txt
@@ -182,7 +182,7 @@ Results [4]: [w_state#14, i_item_id#17, sum(CASE WHEN (d_date#21 < 2000-03-11) T
(33) TakeOrderedAndProject
Input [4]: [w_state#14, i_item_id#17, sales_before#33, sales_after#34]
-Arguments: 100, 0, [w_state#14 ASC NULLS FIRST, i_item_id#17 ASC NULLS FIRST], [w_state#14, i_item_id#17, sales_before#33, sales_after#34]
+Arguments: 100, [w_state#14 ASC NULLS FIRST, i_item_id#17 ASC NULLS FIRST], [w_state#14, i_item_id#17, sales_before#33, sales_after#34]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.sf100/explain.txt
index 6b7de9ade3b00..33c03d2b767dd 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.sf100/explain.txt
@@ -116,5 +116,5 @@ Results [1]: [i_product_name#3]
(20) TakeOrderedAndProject
Input [1]: [i_product_name#3]
-Arguments: 100, 0, [i_product_name#3 ASC NULLS FIRST], [i_product_name#3]
+Arguments: 100, [i_product_name#3 ASC NULLS FIRST], [i_product_name#3]
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/explain.txt
index 6b7de9ade3b00..33c03d2b767dd 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/explain.txt
@@ -116,5 +116,5 @@ Results [1]: [i_product_name#3]
(20) TakeOrderedAndProject
Input [1]: [i_product_name#3]
-Arguments: 100, 0, [i_product_name#3 ASC NULLS FIRST], [i_product_name#3]
+Arguments: 100, [i_product_name#3 ASC NULLS FIRST], [i_product_name#3]
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42.sf100/explain.txt
index c2b578019b9d2..30bd5d90995f4 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42.sf100/explain.txt
@@ -96,7 +96,7 @@ Results [4]: [d_year#11, i_category_id#6, i_category#7, MakeDecimal(sum(Unscaled
(17) TakeOrderedAndProject
Input [4]: [d_year#11, i_category_id#6, i_category#7, sum(ss_ext_sales_price)#16]
-Arguments: 100, 0, [sum(ss_ext_sales_price)#16 DESC NULLS LAST, d_year#11 ASC NULLS FIRST, i_category_id#6 ASC NULLS FIRST, i_category#7 ASC NULLS FIRST], [d_year#11, i_category_id#6, i_category#7, sum(ss_ext_sales_price)#16]
+Arguments: 100, [sum(ss_ext_sales_price)#16 DESC NULLS LAST, d_year#11 ASC NULLS FIRST, i_category_id#6 ASC NULLS FIRST, i_category#7 ASC NULLS FIRST], [d_year#11, i_category_id#6, i_category#7, sum(ss_ext_sales_price)#16]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42/explain.txt
index 964ae1a5ee67f..d16b4bca7c3f4 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42/explain.txt
@@ -119,5 +119,5 @@ Results [4]: [d_year#2, i_category_id#9, i_category#10, MakeDecimal(sum(Unscaled
(21) TakeOrderedAndProject
Input [4]: [d_year#2, i_category_id#9, i_category#10, sum(ss_ext_sales_price)#17]
-Arguments: 100, 0, [sum(ss_ext_sales_price)#17 DESC NULLS LAST, d_year#2 ASC NULLS FIRST, i_category_id#9 ASC NULLS FIRST, i_category#10 ASC NULLS FIRST], [d_year#2, i_category_id#9, i_category#10, sum(ss_ext_sales_price)#17]
+Arguments: 100, [sum(ss_ext_sales_price)#17 DESC NULLS LAST, d_year#2 ASC NULLS FIRST, i_category_id#9 ASC NULLS FIRST, i_category#10 ASC NULLS FIRST], [d_year#2, i_category_id#9, i_category#10, sum(ss_ext_sales_price)#17]
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43.sf100/explain.txt
index 109dc09b759a5..d626333970af9 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43.sf100/explain.txt
@@ -119,7 +119,7 @@ Results [9]: [s_store_name#11, s_store_id#10, MakeDecimal(sum(UnscaledValue(CASE
(21) TakeOrderedAndProject
Input [9]: [s_store_name#11, s_store_id#10, sun_sales#36, mon_sales#37, tue_sales#38, wed_sales#39, thu_sales#40, fri_sales#41, sat_sales#42]
-Arguments: 100, 0, [s_store_name#11 ASC NULLS FIRST, s_store_id#10 ASC NULLS FIRST, sun_sales#36 ASC NULLS FIRST, mon_sales#37 ASC NULLS FIRST, tue_sales#38 ASC NULLS FIRST, wed_sales#39 ASC NULLS FIRST, thu_sales#40 ASC NULLS FIRST, fri_sales#41 ASC NULLS FIRST, sat_sales#42 ASC NULLS FIRST], [s_store_name#11, s_store_id#10, sun_sales#36, mon_sales#37, tue_sales#38, wed_sales#39, thu_sales#40, fri_sales#41, sat_sales#42]
+Arguments: 100, [s_store_name#11 ASC NULLS FIRST, s_store_id#10 ASC NULLS FIRST, sun_sales#36 ASC NULLS FIRST, mon_sales#37 ASC NULLS FIRST, tue_sales#38 ASC NULLS FIRST, wed_sales#39 ASC NULLS FIRST, thu_sales#40 ASC NULLS FIRST, fri_sales#41 ASC NULLS FIRST, sat_sales#42 ASC NULLS FIRST], [s_store_name#11, s_store_id#10, sun_sales#36, mon_sales#37, tue_sales#38, wed_sales#39, thu_sales#40, fri_sales#41, sat_sales#42]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43/explain.txt
index 995c7e32cb6b9..24af5d2e813cb 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43/explain.txt
@@ -119,5 +119,5 @@ Results [9]: [s_store_name#10, s_store_id#9, MakeDecimal(sum(UnscaledValue(CASE
(21) TakeOrderedAndProject
Input [9]: [s_store_name#10, s_store_id#9, sun_sales#35, mon_sales#36, tue_sales#37, wed_sales#38, thu_sales#39, fri_sales#40, sat_sales#41]
-Arguments: 100, 0, [s_store_name#10 ASC NULLS FIRST, s_store_id#9 ASC NULLS FIRST, sun_sales#35 ASC NULLS FIRST, mon_sales#36 ASC NULLS FIRST, tue_sales#37 ASC NULLS FIRST, wed_sales#38 ASC NULLS FIRST, thu_sales#39 ASC NULLS FIRST, fri_sales#40 ASC NULLS FIRST, sat_sales#41 ASC NULLS FIRST], [s_store_name#10, s_store_id#9, sun_sales#35, mon_sales#36, tue_sales#37, wed_sales#38, thu_sales#39, fri_sales#40, sat_sales#41]
+Arguments: 100, [s_store_name#10 ASC NULLS FIRST, s_store_id#9 ASC NULLS FIRST, sun_sales#35 ASC NULLS FIRST, mon_sales#36 ASC NULLS FIRST, tue_sales#37 ASC NULLS FIRST, wed_sales#38 ASC NULLS FIRST, thu_sales#39 ASC NULLS FIRST, fri_sales#40 ASC NULLS FIRST, sat_sales#41 ASC NULLS FIRST], [s_store_name#10, s_store_id#9, sun_sales#35, mon_sales#36, tue_sales#37, wed_sales#38, thu_sales#39, fri_sales#40, sat_sales#41]
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 2b4e326ed9944..e948e6ab3870f 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
@@ -165,7 +165,7 @@ Input [5]: [rnk#16, item_sk#17, i_product_name#22, i_item_sk#24, i_product_name#
(31) TakeOrderedAndProject
Input [3]: [rnk#16, best_performing#26, worst_performing#27]
-Arguments: 100, 0, [rnk#16 ASC NULLS FIRST], [rnk#16, best_performing#26, worst_performing#27]
+Arguments: 100, [rnk#16 ASC NULLS FIRST], [rnk#16, best_performing#26, worst_performing#27]
===== Subqueries =====
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 69bfc9731853f..fc10739c78c78 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
@@ -170,7 +170,7 @@ Input [5]: [rnk#16, item_sk#17, i_product_name#21, i_item_sk#23, i_product_name#
(32) TakeOrderedAndProject
Input [3]: [rnk#16, best_performing#25, worst_performing#26]
-Arguments: 100, 0, [rnk#16 ASC NULLS FIRST], [rnk#16, best_performing#25, worst_performing#26]
+Arguments: 100, [rnk#16 ASC NULLS FIRST], [rnk#16, best_performing#25, worst_performing#26]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.sf100/explain.txt
index 65644947772d8..d5a234a920211 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.sf100/explain.txt
@@ -230,7 +230,7 @@ Results [3]: [ca_zip#17, ca_city#16, MakeDecimal(sum(UnscaledValue(ws_sales_pric
(42) TakeOrderedAndProject
Input [3]: [ca_zip#17, ca_city#16, sum(ws_sales_price)#27]
-Arguments: 100, 0, [ca_zip#17 ASC NULLS FIRST, ca_city#16 ASC NULLS FIRST], [ca_zip#17, ca_city#16, sum(ws_sales_price)#27]
+Arguments: 100, [ca_zip#17 ASC NULLS FIRST, ca_city#16 ASC NULLS FIRST], [ca_zip#17, ca_city#16, sum(ws_sales_price)#27]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/explain.txt
index 02a95e1d674ff..80f9379309580 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/explain.txt
@@ -200,7 +200,7 @@ Results [3]: [ca_zip#12, ca_city#11, MakeDecimal(sum(UnscaledValue(ws_sales_pric
(36) TakeOrderedAndProject
Input [3]: [ca_zip#12, ca_city#11, sum(ws_sales_price)#25]
-Arguments: 100, 0, [ca_zip#12 ASC NULLS FIRST, ca_city#11 ASC NULLS FIRST], [ca_zip#12, ca_city#11, sum(ws_sales_price)#25]
+Arguments: 100, [ca_zip#12 ASC NULLS FIRST, ca_city#11 ASC NULLS FIRST], [ca_zip#12, ca_city#11, sum(ws_sales_price)#25]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.sf100/explain.txt
index e0bb894b147ee..07017d1b64697 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.sf100/explain.txt
@@ -255,7 +255,7 @@ Input [9]: [c_customer_sk#1, c_first_name#3, c_last_name#4, ca_city#7, ss_ticket
(47) TakeOrderedAndProject
Input [7]: [c_last_name#4, c_first_name#3, ca_city#7, bought_city#36, ss_ticket_number#14, amt#37, profit#38]
-Arguments: 100, 0, [c_last_name#4 ASC NULLS FIRST, c_first_name#3 ASC NULLS FIRST, ca_city#7 ASC NULLS FIRST, bought_city#36 ASC NULLS FIRST, ss_ticket_number#14 ASC NULLS FIRST], [c_last_name#4, c_first_name#3, ca_city#7, bought_city#36, ss_ticket_number#14, amt#37, profit#38]
+Arguments: 100, [c_last_name#4 ASC NULLS FIRST, c_first_name#3 ASC NULLS FIRST, ca_city#7 ASC NULLS FIRST, bought_city#36 ASC NULLS FIRST, ss_ticket_number#14 ASC NULLS FIRST], [c_last_name#4, c_first_name#3, ca_city#7, bought_city#36, ss_ticket_number#14, amt#37, profit#38]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/explain.txt
index 490c675dbcc0b..879f2f2ed51d7 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/explain.txt
@@ -215,7 +215,7 @@ Input [9]: [ss_ticket_number#5, bought_city#28, amt#29, profit#30, c_current_add
(39) TakeOrderedAndProject
Input [7]: [c_last_name#34, c_first_name#33, ca_city#37, bought_city#28, ss_ticket_number#5, amt#29, profit#30]
-Arguments: 100, 0, [c_last_name#34 ASC NULLS FIRST, c_first_name#33 ASC NULLS FIRST, ca_city#37 ASC NULLS FIRST, bought_city#28 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#34, c_first_name#33, ca_city#37, bought_city#28, ss_ticket_number#5, amt#29, profit#30]
+Arguments: 100, [c_last_name#34 ASC NULLS FIRST, c_first_name#33 ASC NULLS FIRST, ca_city#37 ASC NULLS FIRST, bought_city#28 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#34, c_first_name#33, ca_city#37, bought_city#28, ss_ticket_number#5, amt#29, profit#30]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.sf100/explain.txt
index e8776e863f73a..ad62e87b8c217 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.sf100/explain.txt
@@ -277,7 +277,7 @@ Input [16]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_ye
(52) TakeOrderedAndProject
Input [10]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#22, psum#49, nsum#50]
-Arguments: 100, 0, [(sum_sales#22 - avg_monthly_sales#26) ASC NULLS FIRST, s_store_name#10 ASC NULLS FIRST], [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#22, psum#49, nsum#50]
+Arguments: 100, [(sum_sales#22 - avg_monthly_sales#26) ASC NULLS FIRST, s_store_name#10 ASC NULLS FIRST], [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#22, psum#49, nsum#50]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/explain.txt
index 3b92d180a08f7..ad8d4176ea19b 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/explain.txt
@@ -242,7 +242,7 @@ Input [16]: [i_category#3, i_brand#2, s_store_name#14, s_company_name#15, d_year
(45) TakeOrderedAndProject
Input [10]: [i_category#3, i_brand#2, s_store_name#14, s_company_name#15, d_year#11, d_moy#12, avg_monthly_sales#25, sum_sales#21, psum#47, nsum#48]
-Arguments: 100, 0, [(sum_sales#21 - avg_monthly_sales#25) ASC NULLS FIRST, s_store_name#14 ASC NULLS FIRST], [i_category#3, i_brand#2, s_store_name#14, s_company_name#15, d_year#11, d_moy#12, avg_monthly_sales#25, sum_sales#21, psum#47, nsum#48]
+Arguments: 100, [(sum_sales#21 - avg_monthly_sales#25) ASC NULLS FIRST, s_store_name#14 ASC NULLS FIRST], [i_category#3, i_brand#2, s_store_name#14, s_company_name#15, d_year#11, d_moy#12, avg_monthly_sales#25, sum_sales#21, psum#47, nsum#48]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.sf100/explain.txt
index 22a678796e2b0..cb5e0c4483205 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.sf100/explain.txt
@@ -469,7 +469,7 @@ Results [5]: [channel#39, item#33, return_ratio#34, return_rank#37, currency_ran
(86) TakeOrderedAndProject
Input [5]: [channel#39, item#33, return_ratio#34, return_rank#37, currency_rank#38]
-Arguments: 100, 0, [channel#39 ASC NULLS FIRST, return_rank#37 ASC NULLS FIRST, currency_rank#38 ASC NULLS FIRST], [channel#39, item#33, return_ratio#34, return_rank#37, currency_rank#38]
+Arguments: 100, [channel#39 ASC NULLS FIRST, return_rank#37 ASC NULLS FIRST, currency_rank#38 ASC NULLS FIRST], [channel#39, item#33, return_ratio#34, return_rank#37, currency_rank#38]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/explain.txt
index 68950e5800e49..be0ce28eeae6b 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/explain.txt
@@ -424,7 +424,7 @@ Results [5]: [channel#38, item#32, return_ratio#33, return_rank#36, currency_ran
(77) TakeOrderedAndProject
Input [5]: [channel#38, item#32, return_ratio#33, return_rank#36, currency_rank#37]
-Arguments: 100, 0, [channel#38 ASC NULLS FIRST, return_rank#36 ASC NULLS FIRST, currency_rank#37 ASC NULLS FIRST], [channel#38, item#32, return_ratio#33, return_rank#36, currency_rank#37]
+Arguments: 100, [channel#38 ASC NULLS FIRST, return_rank#36 ASC NULLS FIRST, currency_rank#37 ASC NULLS FIRST], [channel#38, item#32, return_ratio#33, return_rank#36, currency_rank#37]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/explain.txt
index 88baa0df1062e..3d51214fccab0 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/explain.txt
@@ -429,7 +429,7 @@ Results [5]: [channel#135, id#136, sum(sales#39)#151 AS sales#154, sum(returns#4
(77) TakeOrderedAndProject
Input [5]: [channel#135, id#136, sales#154, returns#155, profit#156]
-Arguments: 100, 0, [channel#135 ASC NULLS FIRST, id#136 ASC NULLS FIRST], [channel#135, id#136, sales#154, returns#155, profit#156]
+Arguments: 100, [channel#135 ASC NULLS FIRST, id#136 ASC NULLS FIRST], [channel#135, id#136, sales#154, returns#155, profit#156]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt
index fd07c6e30afe8..41b16be22cf0d 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt
@@ -414,7 +414,7 @@ Results [5]: [channel#134, id#135, sum(sales#39)#150 AS sales#153, sum(returns#4
(74) TakeOrderedAndProject
Input [5]: [channel#134, id#135, sales#153, returns#154, profit#155]
-Arguments: 100, 0, [channel#134 ASC NULLS FIRST, id#135 ASC NULLS FIRST], [channel#134, id#135, sales#153, returns#154, profit#155]
+Arguments: 100, [channel#134 ASC NULLS FIRST, id#135 ASC NULLS FIRST], [channel#134, id#135, sales#153, returns#154, profit#155]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.sf100/explain.txt
index 037eb09915247..97f8d567853e7 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.sf100/explain.txt
@@ -173,7 +173,7 @@ Results [15]: [s_store_name#17, s_company_id#18, s_street_number#19, s_street_na
(31) TakeOrderedAndProject
Input [15]: [s_store_name#17, s_company_id#18, s_street_number#19, s_street_name#20, s_street_type#21, s_suite_number#22, s_city#23, s_county#24, s_state#25, s_zip#26, 30 days #44, 31 - 60 days #45, 61 - 90 days #46, 91 - 120 days #47, >120 days #48]
-Arguments: 100, 0, [s_store_name#17 ASC NULLS FIRST, s_company_id#18 ASC NULLS FIRST, s_street_number#19 ASC NULLS FIRST, s_street_name#20 ASC NULLS FIRST, s_street_type#21 ASC NULLS FIRST, s_suite_number#22 ASC NULLS FIRST, s_city#23 ASC NULLS FIRST, s_county#24 ASC NULLS FIRST, s_state#25 ASC NULLS FIRST, s_zip#26 ASC NULLS FIRST], [s_store_name#17, s_company_id#18, s_street_number#19, s_street_name#20, s_street_type#21, s_suite_number#22, s_city#23, s_county#24, s_state#25, s_zip#26, 30 days #44, 31 - 60 days #45, 61 - 90 days #46, 91 - 120 days #47, >120 days #48]
+Arguments: 100, [s_store_name#17 ASC NULLS FIRST, s_company_id#18 ASC NULLS FIRST, s_street_number#19 ASC NULLS FIRST, s_street_name#20 ASC NULLS FIRST, s_street_type#21 ASC NULLS FIRST, s_suite_number#22 ASC NULLS FIRST, s_city#23 ASC NULLS FIRST, s_county#24 ASC NULLS FIRST, s_state#25 ASC NULLS FIRST, s_zip#26 ASC NULLS FIRST], [s_store_name#17, s_company_id#18, s_street_number#19, s_street_name#20, s_street_type#21, s_suite_number#22, s_city#23, s_county#24, s_state#25, s_zip#26, 30 days #44, 31 - 60 days #45, 61 - 90 days #46, 91 - 120 days #47, >120 days #48]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50/explain.txt
index 881431216c339..060953c80f295 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50/explain.txt
@@ -158,7 +158,7 @@ Results [15]: [s_store_name#13, s_company_id#14, s_street_number#15, s_street_na
(28) TakeOrderedAndProject
Input [15]: [s_store_name#13, s_company_id#14, s_street_number#15, s_street_name#16, s_street_type#17, s_suite_number#18, s_city#19, s_county#20, s_state#21, s_zip#22, 30 days #43, 31 - 60 days #44, 61 - 90 days #45, 91 - 120 days #46, >120 days #47]
-Arguments: 100, 0, [s_store_name#13 ASC NULLS FIRST, s_company_id#14 ASC NULLS FIRST, s_street_number#15 ASC NULLS FIRST, s_street_name#16 ASC NULLS FIRST, s_street_type#17 ASC NULLS FIRST, s_suite_number#18 ASC NULLS FIRST, s_city#19 ASC NULLS FIRST, s_county#20 ASC NULLS FIRST, s_state#21 ASC NULLS FIRST, s_zip#22 ASC NULLS FIRST], [s_store_name#13, s_company_id#14, s_street_number#15, s_street_name#16, s_street_type#17, s_suite_number#18, s_city#19, s_county#20, s_state#21, s_zip#22, 30 days #43, 31 - 60 days #44, 61 - 90 days #45, 91 - 120 days #46, >120 days #47]
+Arguments: 100, [s_store_name#13 ASC NULLS FIRST, s_company_id#14 ASC NULLS FIRST, s_street_number#15 ASC NULLS FIRST, s_street_name#16 ASC NULLS FIRST, s_street_type#17 ASC NULLS FIRST, s_suite_number#18 ASC NULLS FIRST, s_city#19 ASC NULLS FIRST, s_county#20 ASC NULLS FIRST, s_state#21 ASC NULLS FIRST, s_zip#22 ASC NULLS FIRST], [s_store_name#13, s_company_id#14, s_street_number#15, s_street_name#16, s_street_type#17, s_suite_number#18, s_city#19, s_county#20, s_state#21, s_zip#22, 30 days #43, 31 - 60 days #44, 61 - 90 days #45, 91 - 120 days #46, >120 days #47]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.sf100/explain.txt
index 87459c7075f73..51b1ae5837e70 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.sf100/explain.txt
@@ -203,7 +203,7 @@ Condition : ((isnotnull(web_cumulative#35) AND isnotnull(store_cumulative#36)) A
(37) TakeOrderedAndProject
Input [6]: [item_sk#30, d_date#31, web_sales#32, store_sales#33, web_cumulative#35, store_cumulative#36]
-Arguments: 100, 0, [item_sk#30 ASC NULLS FIRST, d_date#31 ASC NULLS FIRST], [item_sk#30, d_date#31, web_sales#32, store_sales#33, web_cumulative#35, store_cumulative#36]
+Arguments: 100, [item_sk#30 ASC NULLS FIRST, d_date#31 ASC NULLS FIRST], [item_sk#30, d_date#31, web_sales#32, store_sales#33, web_cumulative#35, store_cumulative#36]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/explain.txt
index 87459c7075f73..51b1ae5837e70 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/explain.txt
@@ -203,7 +203,7 @@ Condition : ((isnotnull(web_cumulative#35) AND isnotnull(store_cumulative#36)) A
(37) TakeOrderedAndProject
Input [6]: [item_sk#30, d_date#31, web_sales#32, store_sales#33, web_cumulative#35, store_cumulative#36]
-Arguments: 100, 0, [item_sk#30 ASC NULLS FIRST, d_date#31 ASC NULLS FIRST], [item_sk#30, d_date#31, web_sales#32, store_sales#33, web_cumulative#35, store_cumulative#36]
+Arguments: 100, [item_sk#30 ASC NULLS FIRST, d_date#31 ASC NULLS FIRST], [item_sk#30, d_date#31, web_sales#32, store_sales#33, web_cumulative#35, store_cumulative#36]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52.sf100/explain.txt
index c537870505b24..2c8c5e970052f 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52.sf100/explain.txt
@@ -96,7 +96,7 @@ Results [4]: [d_year#11, i_brand_id#6 AS brand_id#16, i_brand#7 AS brand#17, Mak
(17) TakeOrderedAndProject
Input [4]: [d_year#11, brand_id#16, brand#17, ext_price#18]
-Arguments: 100, 0, [d_year#11 ASC NULLS FIRST, ext_price#18 DESC NULLS LAST, brand_id#16 ASC NULLS FIRST], [d_year#11, brand_id#16, brand#17, ext_price#18]
+Arguments: 100, [d_year#11 ASC NULLS FIRST, ext_price#18 DESC NULLS LAST, brand_id#16 ASC NULLS FIRST], [d_year#11, brand_id#16, brand#17, ext_price#18]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52/explain.txt
index 7c5ef82c9b2f4..e6e106706fb07 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52/explain.txt
@@ -119,5 +119,5 @@ Results [4]: [d_year#2, i_brand_id#9 AS brand_id#17, i_brand#10 AS brand#18, Mak
(21) TakeOrderedAndProject
Input [4]: [d_year#2, brand_id#17, brand#18, ext_price#19]
-Arguments: 100, 0, [d_year#2 ASC NULLS FIRST, ext_price#19 DESC NULLS LAST, brand_id#17 ASC NULLS FIRST], [d_year#2, brand_id#17, brand#18, ext_price#19]
+Arguments: 100, [d_year#2 ASC NULLS FIRST, ext_price#19 DESC NULLS LAST, brand_id#17 ASC NULLS FIRST], [d_year#2, brand_id#17, brand#18, ext_price#19]
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.sf100/explain.txt
index c79f5f4d1020f..475802c40b4ca 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.sf100/explain.txt
@@ -154,7 +154,7 @@ Input [4]: [i_manufact_id#5, sum_sales#24, _w0#25, avg_quarterly_sales#27]
(28) TakeOrderedAndProject
Input [3]: [i_manufact_id#5, sum_sales#24, avg_quarterly_sales#27]
-Arguments: 100, 0, [avg_quarterly_sales#27 ASC NULLS FIRST, sum_sales#24 ASC NULLS FIRST, i_manufact_id#5 ASC NULLS FIRST], [i_manufact_id#5, sum_sales#24, avg_quarterly_sales#27]
+Arguments: 100, [avg_quarterly_sales#27 ASC NULLS FIRST, sum_sales#24 ASC NULLS FIRST, i_manufact_id#5 ASC NULLS FIRST], [i_manufact_id#5, sum_sales#24, avg_quarterly_sales#27]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/explain.txt
index 60c5d8cd45618..0f1b122b830f3 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/explain.txt
@@ -154,7 +154,7 @@ Input [4]: [i_manufact_id#5, sum_sales#24, _w0#25, avg_quarterly_sales#27]
(28) TakeOrderedAndProject
Input [3]: [i_manufact_id#5, sum_sales#24, avg_quarterly_sales#27]
-Arguments: 100, 0, [avg_quarterly_sales#27 ASC NULLS FIRST, sum_sales#24 ASC NULLS FIRST, i_manufact_id#5 ASC NULLS FIRST], [i_manufact_id#5, sum_sales#24, avg_quarterly_sales#27]
+Arguments: 100, [avg_quarterly_sales#27 ASC NULLS FIRST, sum_sales#24 ASC NULLS FIRST, i_manufact_id#5 ASC NULLS FIRST], [i_manufact_id#5, sum_sales#24, avg_quarterly_sales#27]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.sf100/explain.txt
index fe7ebb8dd6fe3..bde0ba3179ca3 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.sf100/explain.txt
@@ -332,7 +332,7 @@ Results [3]: [segment#39, count(1)#43 AS num_customers#44, (segment#39 * 50) AS
(59) TakeOrderedAndProject
Input [3]: [segment#39, num_customers#44, segment_base#45]
-Arguments: 100, 0, [segment#39 ASC NULLS FIRST, num_customers#44 ASC NULLS FIRST], [segment#39, num_customers#44, segment_base#45]
+Arguments: 100, [segment#39 ASC NULLS FIRST, num_customers#44 ASC NULLS FIRST], [segment#39, num_customers#44, segment_base#45]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/explain.txt
index 5d6155e0df030..4e45e0e92f98d 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/explain.txt
@@ -317,7 +317,7 @@ Results [3]: [segment#40, count(1)#44 AS num_customers#45, (segment#40 * 50) AS
(56) TakeOrderedAndProject
Input [3]: [segment#40, num_customers#45, segment_base#46]
-Arguments: 100, 0, [segment#40 ASC NULLS FIRST, num_customers#45 ASC NULLS FIRST], [segment#40, num_customers#45, segment_base#46]
+Arguments: 100, [segment#40 ASC NULLS FIRST, num_customers#45 ASC NULLS FIRST], [segment#40, num_customers#45, segment_base#46]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55.sf100/explain.txt
index 3c89db1fd10c7..c6870bd3ac6a3 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55.sf100/explain.txt
@@ -96,7 +96,7 @@ Results [3]: [i_brand_id#6 AS brand_id#15, i_brand#7 AS brand#16, MakeDecimal(su
(17) TakeOrderedAndProject
Input [3]: [brand_id#15, brand#16, ext_price#17]
-Arguments: 100, 0, [ext_price#17 DESC NULLS LAST, brand_id#15 ASC NULLS FIRST], [brand_id#15, brand#16, ext_price#17]
+Arguments: 100, [ext_price#17 DESC NULLS LAST, brand_id#15 ASC NULLS FIRST], [brand_id#15, brand#16, ext_price#17]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55/explain.txt
index 5caf0d26ff2eb..25b423382e332 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55/explain.txt
@@ -119,5 +119,5 @@ Results [3]: [i_brand_id#9 AS brand_id#17, i_brand#10 AS brand#18, MakeDecimal(s
(21) TakeOrderedAndProject
Input [3]: [brand_id#17, brand#18, ext_price#19]
-Arguments: 100, 0, [ext_price#19 DESC NULLS LAST, brand_id#17 ASC NULLS FIRST], [brand_id#17, brand#18, ext_price#19]
+Arguments: 100, [ext_price#19 DESC NULLS LAST, brand_id#17 ASC NULLS FIRST], [brand_id#17, brand#18, ext_price#19]
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/explain.txt
index 52e46d8fdb345..a7b59fe5d5ea7 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/explain.txt
@@ -354,7 +354,7 @@ Results [2]: [i_item_id#11, sum(total_sales#20)#52 AS total_sales#53]
(63) TakeOrderedAndProject
Input [2]: [i_item_id#11, total_sales#53]
-Arguments: 100, 0, [total_sales#53 ASC NULLS FIRST], [i_item_id#11, total_sales#53]
+Arguments: 100, [total_sales#53 ASC NULLS FIRST], [i_item_id#11, total_sales#53]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/explain.txt
index 52e46d8fdb345..a7b59fe5d5ea7 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/explain.txt
@@ -354,7 +354,7 @@ Results [2]: [i_item_id#11, sum(total_sales#20)#52 AS total_sales#53]
(63) TakeOrderedAndProject
Input [2]: [i_item_id#11, total_sales#53]
-Arguments: 100, 0, [total_sales#53 ASC NULLS FIRST], [i_item_id#11, total_sales#53]
+Arguments: 100, [total_sales#53 ASC NULLS FIRST], [i_item_id#11, total_sales#53]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.sf100/explain.txt
index e6d847deb40ca..03f395cd454cb 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.sf100/explain.txt
@@ -277,7 +277,7 @@ Input [14]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, sum_sales
(52) TakeOrderedAndProject
Input [9]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#21, psum#46, nsum#47]
-Arguments: 100, 0, [(sum_sales#21 - avg_monthly_sales#25) ASC NULLS FIRST, cc_name#10 ASC NULLS FIRST], [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#21, psum#46, nsum#47]
+Arguments: 100, [(sum_sales#21 - avg_monthly_sales#25) ASC NULLS FIRST, cc_name#10 ASC NULLS FIRST], [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#21, psum#46, nsum#47]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/explain.txt
index c566579dcfe50..a2df663e63572 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/explain.txt
@@ -242,7 +242,7 @@ Input [14]: [i_category#3, i_brand#2, cc_name#14, d_year#11, d_moy#12, sum_sales
(45) TakeOrderedAndProject
Input [9]: [i_category#3, i_brand#2, cc_name#14, d_year#11, d_moy#12, avg_monthly_sales#24, sum_sales#20, psum#44, nsum#45]
-Arguments: 100, 0, [(sum_sales#20 - avg_monthly_sales#24) ASC NULLS FIRST, cc_name#14 ASC NULLS FIRST], [i_category#3, i_brand#2, cc_name#14, d_year#11, d_moy#12, avg_monthly_sales#24, sum_sales#20, psum#44, nsum#45]
+Arguments: 100, [(sum_sales#20 - avg_monthly_sales#24) ASC NULLS FIRST, cc_name#14 ASC NULLS FIRST], [i_category#3, i_brand#2, cc_name#14, d_year#11, d_moy#12, avg_monthly_sales#24, sum_sales#20, psum#44, nsum#45]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/explain.txt
index 1ddd7c7bc6679..b573b7a6cc39b 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/explain.txt
@@ -276,7 +276,7 @@ Input [5]: [item_id#13, ss_item_rev#14, cs_item_rev#26, item_id#38, ws_item_rev#
(49) TakeOrderedAndProject
Input [8]: [item_id#13, ss_item_rev#14, ss_dev#41, cs_item_rev#26, cs_dev#42, ws_item_rev#39, ws_dev#43, average#44]
-Arguments: 100, 0, [item_id#13 ASC NULLS FIRST, ss_item_rev#14 ASC NULLS FIRST], [item_id#13, ss_item_rev#14, ss_dev#41, cs_item_rev#26, cs_dev#42, ws_item_rev#39, ws_dev#43, average#44]
+Arguments: 100, [item_id#13 ASC NULLS FIRST, ss_item_rev#14 ASC NULLS FIRST], [item_id#13, ss_item_rev#14, ss_dev#41, cs_item_rev#26, cs_dev#42, ws_item_rev#39, ws_dev#43, average#44]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt
index fc696498b5259..4178025b05292 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt
@@ -276,7 +276,7 @@ Input [5]: [item_id#13, ss_item_rev#14, cs_item_rev#26, item_id#38, ws_item_rev#
(49) TakeOrderedAndProject
Input [8]: [item_id#13, ss_item_rev#14, ss_dev#41, cs_item_rev#26, cs_dev#42, ws_item_rev#39, ws_dev#43, average#44]
-Arguments: 100, 0, [item_id#13 ASC NULLS FIRST, ss_item_rev#14 ASC NULLS FIRST], [item_id#13, ss_item_rev#14, ss_dev#41, cs_item_rev#26, cs_dev#42, ws_item_rev#39, ws_dev#43, average#44]
+Arguments: 100, [item_id#13 ASC NULLS FIRST, ss_item_rev#14 ASC NULLS FIRST], [item_id#13, ss_item_rev#14, ss_dev#41, cs_item_rev#26, cs_dev#42, ws_item_rev#39, ws_dev#43, average#44]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59.sf100/explain.txt
index a433d1c6d0da0..a91072c13c539 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59.sf100/explain.txt
@@ -246,5 +246,5 @@ Input [19]: [s_store_name1#44, d_week_seq1#45, s_store_id1#46, sun_sales1#47, mo
(44) TakeOrderedAndProject
Input [10]: [s_store_name1#44, s_store_id1#46, d_week_seq1#45, (sun_sales1 / sun_sales2)#77, (mon_sales1 / mon_sales2)#78, (tue_sales1 / tue_sales2)#79, (wed_sales1 / wed_sales2)#80, (thu_sales1 / thu_sales2)#81, (fri_sales1 / fri_sales2)#82, (sat_sales1 / sat_sales2)#83]
-Arguments: 100, 0, [s_store_name1#44 ASC NULLS FIRST, s_store_id1#46 ASC NULLS FIRST, d_week_seq1#45 ASC NULLS FIRST], [s_store_name1#44, s_store_id1#46, d_week_seq1#45, (sun_sales1 / sun_sales2)#77, (mon_sales1 / mon_sales2)#78, (tue_sales1 / tue_sales2)#79, (wed_sales1 / wed_sales2)#80, (thu_sales1 / thu_sales2)#81, (fri_sales1 / fri_sales2)#82, (sat_sales1 / sat_sales2)#83]
+Arguments: 100, [s_store_name1#44 ASC NULLS FIRST, s_store_id1#46 ASC NULLS FIRST, d_week_seq1#45 ASC NULLS FIRST], [s_store_name1#44, s_store_id1#46, d_week_seq1#45, (sun_sales1 / sun_sales2)#77, (mon_sales1 / mon_sales2)#78, (tue_sales1 / tue_sales2)#79, (wed_sales1 / wed_sales2)#80, (thu_sales1 / thu_sales2)#81, (fri_sales1 / fri_sales2)#82, (sat_sales1 / sat_sales2)#83]
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/explain.txt
index a433d1c6d0da0..a91072c13c539 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/explain.txt
@@ -246,5 +246,5 @@ Input [19]: [s_store_name1#44, d_week_seq1#45, s_store_id1#46, sun_sales1#47, mo
(44) TakeOrderedAndProject
Input [10]: [s_store_name1#44, s_store_id1#46, d_week_seq1#45, (sun_sales1 / sun_sales2)#77, (mon_sales1 / mon_sales2)#78, (tue_sales1 / tue_sales2)#79, (wed_sales1 / wed_sales2)#80, (thu_sales1 / thu_sales2)#81, (fri_sales1 / fri_sales2)#82, (sat_sales1 / sat_sales2)#83]
-Arguments: 100, 0, [s_store_name1#44 ASC NULLS FIRST, s_store_id1#46 ASC NULLS FIRST, d_week_seq1#45 ASC NULLS FIRST], [s_store_name1#44, s_store_id1#46, d_week_seq1#45, (sun_sales1 / sun_sales2)#77, (mon_sales1 / mon_sales2)#78, (tue_sales1 / tue_sales2)#79, (wed_sales1 / wed_sales2)#80, (thu_sales1 / thu_sales2)#81, (fri_sales1 / fri_sales2)#82, (sat_sales1 / sat_sales2)#83]
+Arguments: 100, [s_store_name1#44 ASC NULLS FIRST, s_store_id1#46 ASC NULLS FIRST, d_week_seq1#45 ASC NULLS FIRST], [s_store_name1#44, s_store_id1#46, d_week_seq1#45, (sun_sales1 / sun_sales2)#77, (mon_sales1 / mon_sales2)#78, (tue_sales1 / tue_sales2)#79, (wed_sales1 / wed_sales2)#80, (thu_sales1 / thu_sales2)#81, (fri_sales1 / fri_sales2)#82, (sat_sales1 / sat_sales2)#83]
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/explain.txt
index 45174940ead37..5ba9cf5ab20b6 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/explain.txt
@@ -354,7 +354,7 @@ Results [2]: [i_item_id#11, sum(total_sales#20)#52 AS total_sales#53]
(63) TakeOrderedAndProject
Input [2]: [i_item_id#11, total_sales#53]
-Arguments: 100, 0, [i_item_id#11 ASC NULLS FIRST, total_sales#53 ASC NULLS FIRST], [i_item_id#11, total_sales#53]
+Arguments: 100, [i_item_id#11 ASC NULLS FIRST, total_sales#53 ASC NULLS FIRST], [i_item_id#11, total_sales#53]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/explain.txt
index 45174940ead37..5ba9cf5ab20b6 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/explain.txt
@@ -354,7 +354,7 @@ Results [2]: [i_item_id#11, sum(total_sales#20)#52 AS total_sales#53]
(63) TakeOrderedAndProject
Input [2]: [i_item_id#11, total_sales#53]
-Arguments: 100, 0, [i_item_id#11 ASC NULLS FIRST, total_sales#53 ASC NULLS FIRST], [i_item_id#11, total_sales#53]
+Arguments: 100, [i_item_id#11 ASC NULLS FIRST, total_sales#53 ASC NULLS FIRST], [i_item_id#11, total_sales#53]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62.sf100/explain.txt
index f43d895b0b0ad..0c82f6182c240 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62.sf100/explain.txt
@@ -179,5 +179,5 @@ Results [8]: [_groupingexpression#18 AS substr(w_warehouse_name, 1, 20)#35, sm_t
(32) TakeOrderedAndProject
Input [8]: [substr(w_warehouse_name, 1, 20)#35, sm_type#13, web_name#10, 30 days #36, 31 - 60 days #37, 61 - 90 days #38, 91 - 120 days #39, >120 days #40]
-Arguments: 100, 0, [substr(w_warehouse_name, 1, 20)#35 ASC NULLS FIRST, sm_type#13 ASC NULLS FIRST, web_name#10 ASC NULLS FIRST], [substr(w_warehouse_name, 1, 20)#35, sm_type#13, web_name#10, 30 days #36, 31 - 60 days #37, 61 - 90 days #38, 91 - 120 days #39, >120 days #40]
+Arguments: 100, [substr(w_warehouse_name, 1, 20)#35 ASC NULLS FIRST, sm_type#13 ASC NULLS FIRST, web_name#10 ASC NULLS FIRST], [substr(w_warehouse_name, 1, 20)#35, sm_type#13, web_name#10, 30 days #36, 31 - 60 days #37, 61 - 90 days #38, 91 - 120 days #39, >120 days #40]
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62/explain.txt
index 7c9e44012153e..752025ebea0a5 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62/explain.txt
@@ -179,5 +179,5 @@ Results [8]: [_groupingexpression#18 AS substr(w_warehouse_name, 1, 20)#35, sm_t
(32) TakeOrderedAndProject
Input [8]: [substr(w_warehouse_name, 1, 20)#35, sm_type#10, web_name#13, 30 days #36, 31 - 60 days #37, 61 - 90 days #38, 91 - 120 days #39, >120 days #40]
-Arguments: 100, 0, [substr(w_warehouse_name, 1, 20)#35 ASC NULLS FIRST, sm_type#10 ASC NULLS FIRST, web_name#13 ASC NULLS FIRST], [substr(w_warehouse_name, 1, 20)#35, sm_type#10, web_name#13, 30 days #36, 31 - 60 days #37, 61 - 90 days #38, 91 - 120 days #39, >120 days #40]
+Arguments: 100, [substr(w_warehouse_name, 1, 20)#35 ASC NULLS FIRST, sm_type#10 ASC NULLS FIRST, web_name#13 ASC NULLS FIRST], [substr(w_warehouse_name, 1, 20)#35, sm_type#10, web_name#13, 30 days #36, 31 - 60 days #37, 61 - 90 days #38, 91 - 120 days #39, >120 days #40]
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.sf100/explain.txt
index 54cd4f6a8b4e5..eb3e3baf14a3a 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.sf100/explain.txt
@@ -154,7 +154,7 @@ Input [4]: [i_manager_id#5, sum_sales#24, _w0#25, avg_monthly_sales#27]
(28) TakeOrderedAndProject
Input [3]: [i_manager_id#5, sum_sales#24, avg_monthly_sales#27]
-Arguments: 100, 0, [i_manager_id#5 ASC NULLS FIRST, avg_monthly_sales#27 ASC NULLS FIRST, sum_sales#24 ASC NULLS FIRST], [i_manager_id#5, sum_sales#24, avg_monthly_sales#27]
+Arguments: 100, [i_manager_id#5 ASC NULLS FIRST, avg_monthly_sales#27 ASC NULLS FIRST, sum_sales#24 ASC NULLS FIRST], [i_manager_id#5, sum_sales#24, avg_monthly_sales#27]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/explain.txt
index ca16f6b732e74..a94546e0a7158 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/explain.txt
@@ -154,7 +154,7 @@ Input [4]: [i_manager_id#5, sum_sales#24, _w0#25, avg_monthly_sales#27]
(28) TakeOrderedAndProject
Input [3]: [i_manager_id#5, sum_sales#24, avg_monthly_sales#27]
-Arguments: 100, 0, [i_manager_id#5 ASC NULLS FIRST, avg_monthly_sales#27 ASC NULLS FIRST, sum_sales#24 ASC NULLS FIRST], [i_manager_id#5, sum_sales#24, avg_monthly_sales#27]
+Arguments: 100, [i_manager_id#5 ASC NULLS FIRST, avg_monthly_sales#27 ASC NULLS FIRST, sum_sales#24 ASC NULLS FIRST], [i_manager_id#5, sum_sales#24, avg_monthly_sales#27]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.sf100/explain.txt
index 51f213a961ca0..50016cc6b5e5a 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.sf100/explain.txt
@@ -240,7 +240,7 @@ Input [8]: [ss_item_sk#1, revenue#11, s_store_name#31, i_item_sk#34, i_item_desc
(42) TakeOrderedAndProject
Input [6]: [s_store_name#31, i_item_desc#35, revenue#11, i_current_price#36, i_wholesale_cost#37, i_brand#38]
-Arguments: 100, 0, [s_store_name#31 ASC NULLS FIRST, i_item_desc#35 ASC NULLS FIRST], [s_store_name#31, i_item_desc#35, revenue#11, i_current_price#36, i_wholesale_cost#37, i_brand#38]
+Arguments: 100, [s_store_name#31 ASC NULLS FIRST, i_item_desc#35 ASC NULLS FIRST], [s_store_name#31, i_item_desc#35, revenue#11, i_current_price#36, i_wholesale_cost#37, i_brand#38]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/explain.txt
index 38166fcd0970b..458daedc8f2ca 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/explain.txt
@@ -225,7 +225,7 @@ Input [9]: [s_store_name#2, ss_store_sk#4, revenue#13, i_item_desc#16, i_current
(39) TakeOrderedAndProject
Input [6]: [s_store_name#2, i_item_desc#16, revenue#13, i_current_price#17, i_wholesale_cost#18, i_brand#19]
-Arguments: 100, 0, [s_store_name#2 ASC NULLS FIRST, i_item_desc#16 ASC NULLS FIRST], [s_store_name#2, i_item_desc#16, revenue#13, i_current_price#17, i_wholesale_cost#18, i_brand#19]
+Arguments: 100, [s_store_name#2 ASC NULLS FIRST, i_item_desc#16 ASC NULLS FIRST], [s_store_name#2, i_item_desc#16, revenue#13, i_current_price#17, i_wholesale_cost#18, i_brand#19]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.sf100/explain.txt
index 4e2b84fbd5f59..f26c4cb712bc4 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.sf100/explain.txt
@@ -290,7 +290,7 @@ Results [44]: [w_warehouse_name#19, w_warehouse_sq_ft#20, w_city#21, w_county#22
(52) TakeOrderedAndProject
Input [44]: [w_warehouse_name#19, w_warehouse_sq_ft#20, w_city#21, w_county#22, w_state#23, w_country#24, ship_carriers#147, year#148, jan_sales#520, feb_sales#521, mar_sales#522, apr_sales#523, may_sales#524, jun_sales#525, jul_sales#526, aug_sales#527, sep_sales#528, oct_sales#529, nov_sales#530, dec_sales#531, jan_sales_per_sq_foot#532, feb_sales_per_sq_foot#533, mar_sales_per_sq_foot#534, apr_sales_per_sq_foot#535, may_sales_per_sq_foot#536, jun_sales_per_sq_foot#537, jul_sales_per_sq_foot#538, aug_sales_per_sq_foot#539, sep_sales_per_sq_foot#540, oct_sales_per_sq_foot#541, nov_sales_per_sq_foot#542, dec_sales_per_sq_foot#543, jan_net#544, feb_net#545, mar_net#546, apr_net#547, may_net#548, jun_net#549, jul_net#550, aug_net#551, sep_net#552, oct_net#553, nov_net#554, dec_net#555]
-Arguments: 100, 0, [w_warehouse_name#19 ASC NULLS FIRST], [w_warehouse_name#19, w_warehouse_sq_ft#20, w_city#21, w_county#22, w_state#23, w_country#24, ship_carriers#147, year#148, jan_sales#520, feb_sales#521, mar_sales#522, apr_sales#523, may_sales#524, jun_sales#525, jul_sales#526, aug_sales#527, sep_sales#528, oct_sales#529, nov_sales#530, dec_sales#531, jan_sales_per_sq_foot#532, feb_sales_per_sq_foot#533, mar_sales_per_sq_foot#534, apr_sales_per_sq_foot#535, ... 20 more fields]
+Arguments: 100, [w_warehouse_name#19 ASC NULLS FIRST], [w_warehouse_name#19, w_warehouse_sq_ft#20, w_city#21, w_county#22, w_state#23, w_country#24, ship_carriers#147, year#148, jan_sales#520, feb_sales#521, mar_sales#522, apr_sales#523, may_sales#524, jun_sales#525, jul_sales#526, aug_sales#527, sep_sales#528, oct_sales#529, nov_sales#530, dec_sales#531, jan_sales_per_sq_foot#532, feb_sales_per_sq_foot#533, mar_sales_per_sq_foot#534, apr_sales_per_sq_foot#535, ... 20 more fields]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/explain.txt
index 75cb3039d13f8..b2f34251c33ca 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/explain.txt
@@ -290,7 +290,7 @@ Results [44]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13
(52) TakeOrderedAndProject
Input [44]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#147, year#148, jan_sales#520, feb_sales#521, mar_sales#522, apr_sales#523, may_sales#524, jun_sales#525, jul_sales#526, aug_sales#527, sep_sales#528, oct_sales#529, nov_sales#530, dec_sales#531, jan_sales_per_sq_foot#532, feb_sales_per_sq_foot#533, mar_sales_per_sq_foot#534, apr_sales_per_sq_foot#535, may_sales_per_sq_foot#536, jun_sales_per_sq_foot#537, jul_sales_per_sq_foot#538, aug_sales_per_sq_foot#539, sep_sales_per_sq_foot#540, oct_sales_per_sq_foot#541, nov_sales_per_sq_foot#542, dec_sales_per_sq_foot#543, jan_net#544, feb_net#545, mar_net#546, apr_net#547, may_net#548, jun_net#549, jul_net#550, aug_net#551, sep_net#552, oct_net#553, nov_net#554, dec_net#555]
-Arguments: 100, 0, [w_warehouse_name#10 ASC NULLS FIRST], [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#147, year#148, jan_sales#520, feb_sales#521, mar_sales#522, apr_sales#523, may_sales#524, jun_sales#525, jul_sales#526, aug_sales#527, sep_sales#528, oct_sales#529, nov_sales#530, dec_sales#531, jan_sales_per_sq_foot#532, feb_sales_per_sq_foot#533, mar_sales_per_sq_foot#534, apr_sales_per_sq_foot#535, ... 20 more fields]
+Arguments: 100, [w_warehouse_name#10 ASC NULLS FIRST], [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#147, year#148, jan_sales#520, feb_sales#521, mar_sales#522, apr_sales#523, may_sales#524, jun_sales#525, jul_sales#526, aug_sales#527, sep_sales#528, oct_sales#529, nov_sales#530, dec_sales#531, jan_sales_per_sq_foot#532, feb_sales_per_sq_foot#533, mar_sales_per_sq_foot#534, apr_sales_per_sq_foot#535, ... 20 more fields]
===== Subqueries =====
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 6a2556b1ca700..8c571092e8807 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
@@ -164,7 +164,7 @@ Condition : (rk#38 <= 100)
(30) TakeOrderedAndProject
Input [10]: [i_category#21, i_class#22, i_brand#23, i_product_name#24, d_year#25, d_qoy#26, d_moy#27, s_store_id#28, sumsales#36, rk#38]
-Arguments: 100, 0, [i_category#21 ASC NULLS FIRST, i_class#22 ASC NULLS FIRST, i_brand#23 ASC NULLS FIRST, i_product_name#24 ASC NULLS FIRST, d_year#25 ASC NULLS FIRST, d_qoy#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST, s_store_id#28 ASC NULLS FIRST, sumsales#36 ASC NULLS FIRST, rk#38 ASC NULLS FIRST], [i_category#21, i_class#22, i_brand#23, i_product_name#24, d_year#25, d_qoy#26, d_moy#27, s_store_id#28, sumsales#36, rk#38]
+Arguments: 100, [i_category#21 ASC NULLS FIRST, i_class#22 ASC NULLS FIRST, i_brand#23 ASC NULLS FIRST, i_product_name#24 ASC NULLS FIRST, d_year#25 ASC NULLS FIRST, d_qoy#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST, s_store_id#28 ASC NULLS FIRST, sumsales#36 ASC NULLS FIRST, rk#38 ASC NULLS FIRST], [i_category#21, i_class#22, i_brand#23, i_product_name#24, d_year#25, d_qoy#26, d_moy#27, s_store_id#28, sumsales#36, rk#38]
===== Subqueries =====
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 2b73a21d0c1ce..83017e0531c4b 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
@@ -149,7 +149,7 @@ Condition : (rk#37 <= 100)
(27) TakeOrderedAndProject
Input [10]: [i_category#20, i_class#21, i_brand#22, i_product_name#23, d_year#24, d_qoy#25, d_moy#26, s_store_id#27, sumsales#35, rk#37]
-Arguments: 100, 0, [i_category#20 ASC NULLS FIRST, i_class#21 ASC NULLS FIRST, i_brand#22 ASC NULLS FIRST, i_product_name#23 ASC NULLS FIRST, d_year#24 ASC NULLS FIRST, d_qoy#25 ASC NULLS FIRST, d_moy#26 ASC NULLS FIRST, s_store_id#27 ASC NULLS FIRST, sumsales#35 ASC NULLS FIRST, rk#37 ASC NULLS FIRST], [i_category#20, i_class#21, i_brand#22, i_product_name#23, d_year#24, d_qoy#25, d_moy#26, s_store_id#27, sumsales#35, rk#37]
+Arguments: 100, [i_category#20 ASC NULLS FIRST, i_class#21 ASC NULLS FIRST, i_brand#22 ASC NULLS FIRST, i_product_name#23 ASC NULLS FIRST, d_year#24 ASC NULLS FIRST, d_qoy#25 ASC NULLS FIRST, d_moy#26 ASC NULLS FIRST, s_store_id#27 ASC NULLS FIRST, sumsales#35 ASC NULLS FIRST, rk#37 ASC NULLS FIRST], [i_category#20, i_class#21, i_brand#22, i_product_name#23, d_year#24, d_qoy#25, d_moy#26, s_store_id#27, sumsales#35, rk#37]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.sf100/explain.txt
index ce8cb45f8cc4b..2075dff70a542 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.sf100/explain.txt
@@ -255,7 +255,7 @@ Input [10]: [c_customer_sk#1, c_first_name#3, c_last_name#4, ca_city#7, ss_ticke
(47) TakeOrderedAndProject
Input [8]: [c_last_name#4, c_first_name#3, ca_city#7, bought_city#40, ss_ticket_number#14, extended_price#41, extended_tax#43, list_price#42]
-Arguments: 100, 0, [c_last_name#4 ASC NULLS FIRST, ss_ticket_number#14 ASC NULLS FIRST], [c_last_name#4, c_first_name#3, ca_city#7, bought_city#40, ss_ticket_number#14, extended_price#41, extended_tax#43, list_price#42]
+Arguments: 100, [c_last_name#4 ASC NULLS FIRST, ss_ticket_number#14 ASC NULLS FIRST], [c_last_name#4, c_first_name#3, ca_city#7, bought_city#40, ss_ticket_number#14, extended_price#41, extended_tax#43, list_price#42]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/explain.txt
index bb7750a187793..40336e3e7c909 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/explain.txt
@@ -215,7 +215,7 @@ Input [10]: [ss_ticket_number#5, bought_city#32, extended_price#33, list_price#3
(39) TakeOrderedAndProject
Input [8]: [c_last_name#39, c_first_name#38, ca_city#42, bought_city#32, ss_ticket_number#5, extended_price#33, extended_tax#35, list_price#34]
-Arguments: 100, 0, [c_last_name#39 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#39, c_first_name#38, ca_city#42, bought_city#32, ss_ticket_number#5, extended_price#33, extended_tax#35, list_price#34]
+Arguments: 100, [c_last_name#39 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#39, c_first_name#38, ca_city#42, bought_city#32, ss_ticket_number#5, extended_price#33, extended_tax#35, list_price#34]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.sf100/explain.txt
index 8294ae86b9b44..339d7d013972d 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.sf100/explain.txt
@@ -257,7 +257,7 @@ Results [8]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, count(
(47) TakeOrderedAndProject
Input [8]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cnt1#32, cd_purchase_estimate#26, cnt2#33, cd_credit_rating#27, cnt3#34]
-Arguments: 100, 0, [cd_gender#23 ASC NULLS FIRST, cd_marital_status#24 ASC NULLS FIRST, cd_education_status#25 ASC NULLS FIRST, cd_purchase_estimate#26 ASC NULLS FIRST, cd_credit_rating#27 ASC NULLS FIRST], [cd_gender#23, cd_marital_status#24, cd_education_status#25, cnt1#32, cd_purchase_estimate#26, cnt2#33, cd_credit_rating#27, cnt3#34]
+Arguments: 100, [cd_gender#23 ASC NULLS FIRST, cd_marital_status#24 ASC NULLS FIRST, cd_education_status#25 ASC NULLS FIRST, cd_purchase_estimate#26 ASC NULLS FIRST, cd_credit_rating#27 ASC NULLS FIRST], [cd_gender#23, cd_marital_status#24, cd_education_status#25, cnt1#32, cd_purchase_estimate#26, cnt2#33, cd_credit_rating#27, cnt3#34]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/explain.txt
index a7a96ca9b796c..f509fa18d6971 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/explain.txt
@@ -232,7 +232,7 @@ Results [8]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, count(
(42) TakeOrderedAndProject
Input [8]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cnt1#31, cd_purchase_estimate#24, cnt2#32, cd_credit_rating#25, cnt3#33]
-Arguments: 100, 0, [cd_gender#21 ASC NULLS FIRST, cd_marital_status#22 ASC NULLS FIRST, cd_education_status#23 ASC NULLS FIRST, cd_purchase_estimate#24 ASC NULLS FIRST, cd_credit_rating#25 ASC NULLS FIRST], [cd_gender#21, cd_marital_status#22, cd_education_status#23, cnt1#31, cd_purchase_estimate#24, cnt2#32, cd_credit_rating#25, cnt3#33]
+Arguments: 100, [cd_gender#21 ASC NULLS FIRST, cd_marital_status#22 ASC NULLS FIRST, cd_education_status#23 ASC NULLS FIRST, cd_purchase_estimate#24 ASC NULLS FIRST, cd_credit_rating#25 ASC NULLS FIRST], [cd_gender#21, cd_marital_status#22, cd_education_status#23, cnt1#31, cd_purchase_estimate#24, cnt2#32, cd_credit_rating#25, cnt3#33]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.sf100/explain.txt
index 9e8d299758dc6..129cee0449f6a 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.sf100/explain.txt
@@ -167,7 +167,7 @@ Results [5]: [i_item_id#21, avg(ss_quantity#4)#40 AS agg1#44, cast((avg(Unscaled
(30) TakeOrderedAndProject
Input [5]: [i_item_id#21, agg1#44, agg2#45, agg3#46, agg4#47]
-Arguments: 100, 0, [i_item_id#21 ASC NULLS FIRST], [i_item_id#21, agg1#44, agg2#45, agg3#46, agg4#47]
+Arguments: 100, [i_item_id#21 ASC NULLS FIRST], [i_item_id#21, agg1#44, agg2#45, agg3#46, agg4#47]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/explain.txt
index fff407d94e2ff..d01608cf02add 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/explain.txt
@@ -167,7 +167,7 @@ Results [5]: [i_item_id#17, avg(ss_quantity#4)#40 AS agg1#44, cast((avg(Unscaled
(30) TakeOrderedAndProject
Input [5]: [i_item_id#17, agg1#44, agg2#45, agg3#46, agg4#47]
-Arguments: 100, 0, [i_item_id#17 ASC NULLS FIRST], [i_item_id#17, agg1#44, agg2#45, agg3#46, agg4#47]
+Arguments: 100, [i_item_id#17 ASC NULLS FIRST], [i_item_id#17, agg1#44, agg2#45, agg3#46, agg4#47]
===== Subqueries =====
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 10a14837d7a8b..bb0ceeabca2ac 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
@@ -239,7 +239,7 @@ Input [8]: [total_sum#33, s_state#26, s_county#27, lochierarchy#34, _w1#35, _w2#
(43) TakeOrderedAndProject
Input [5]: [total_sum#33, s_state#26, s_county#27, lochierarchy#34, rank_within_parent#39]
-Arguments: 100, 0, [lochierarchy#34 DESC NULLS LAST, CASE WHEN (lochierarchy#34 = 0) THEN s_state#26 END ASC NULLS FIRST, rank_within_parent#39 ASC NULLS FIRST], [total_sum#33, s_state#26, s_county#27, lochierarchy#34, rank_within_parent#39]
+Arguments: 100, [lochierarchy#34 DESC NULLS LAST, CASE WHEN (lochierarchy#34 = 0) THEN s_state#26 END ASC NULLS FIRST, rank_within_parent#39 ASC NULLS FIRST], [total_sum#33, s_state#26, s_county#27, lochierarchy#34, rank_within_parent#39]
===== Subqueries =====
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 eeef4cb82284e..a358870666530 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
@@ -239,7 +239,7 @@ Input [8]: [total_sum#33, s_state#26, s_county#27, lochierarchy#34, _w1#35, _w2#
(43) TakeOrderedAndProject
Input [5]: [total_sum#33, s_state#26, s_county#27, lochierarchy#34, rank_within_parent#39]
-Arguments: 100, 0, [lochierarchy#34 DESC NULLS LAST, CASE WHEN (lochierarchy#34 = 0) THEN s_state#26 END ASC NULLS FIRST, rank_within_parent#39 ASC NULLS FIRST], [total_sum#33, s_state#26, s_county#27, lochierarchy#34, rank_within_parent#39]
+Arguments: 100, [lochierarchy#34 DESC NULLS LAST, CASE WHEN (lochierarchy#34 = 0) THEN s_state#26 END ASC NULLS FIRST, rank_within_parent#39 ASC NULLS FIRST], [total_sum#33, s_state#26, s_county#27, lochierarchy#34, rank_within_parent#39]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/explain.txt
index f238c13c4c4e7..c6971f3ea904b 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/explain.txt
@@ -383,7 +383,7 @@ Results [6]: [i_item_desc#21, w_warehouse_name#33, d_week_seq#25, count(1)#46 AS
(70) TakeOrderedAndProject
Input [6]: [i_item_desc#21, w_warehouse_name#33, d_week_seq#25, no_promo#47, promo#48, total_cnt#49]
-Arguments: 100, 0, [total_cnt#49 DESC NULLS LAST, i_item_desc#21 ASC NULLS FIRST, w_warehouse_name#33 ASC NULLS FIRST, d_week_seq#25 ASC NULLS FIRST], [i_item_desc#21, w_warehouse_name#33, d_week_seq#25, no_promo#47, promo#48, total_cnt#49]
+Arguments: 100, [total_cnt#49 DESC NULLS LAST, i_item_desc#21 ASC NULLS FIRST, w_warehouse_name#33 ASC NULLS FIRST, d_week_seq#25 ASC NULLS FIRST], [i_item_desc#21, w_warehouse_name#33, d_week_seq#25, no_promo#47, promo#48, total_cnt#49]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/explain.txt
index 150487e6b1052..e8671f012f8dc 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/explain.txt
@@ -386,7 +386,7 @@ Results [6]: [i_item_desc#19, w_warehouse_name#16, d_week_seq#29, count(1)#46 AS
(70) TakeOrderedAndProject
Input [6]: [i_item_desc#19, w_warehouse_name#16, d_week_seq#29, no_promo#47, promo#48, total_cnt#49]
-Arguments: 100, 0, [total_cnt#49 DESC NULLS LAST, i_item_desc#19 ASC NULLS FIRST, w_warehouse_name#16 ASC NULLS FIRST, d_week_seq#29 ASC NULLS FIRST], [i_item_desc#19, w_warehouse_name#16, d_week_seq#29, no_promo#47, promo#48, total_cnt#49]
+Arguments: 100, [total_cnt#49 DESC NULLS LAST, i_item_desc#19 ASC NULLS FIRST, w_warehouse_name#16 ASC NULLS FIRST, d_week_seq#29 ASC NULLS FIRST], [i_item_desc#19, w_warehouse_name#16, d_week_seq#29, no_promo#47, promo#48, total_cnt#49]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76.sf100/explain.txt
index 4f48f14edef2c..9284172139688 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76.sf100/explain.txt
@@ -244,5 +244,5 @@ Results [7]: [channel#12, col_name#13, d_year#6, d_qoy#7, i_category#10, count(1
(44) TakeOrderedAndProject
Input [7]: [channel#12, col_name#13, d_year#6, d_qoy#7, i_category#10, sales_cnt#48, sales_amt#49]
-Arguments: 100, 0, [channel#12 ASC NULLS FIRST, col_name#13 ASC NULLS FIRST, d_year#6 ASC NULLS FIRST, d_qoy#7 ASC NULLS FIRST, i_category#10 ASC NULLS FIRST], [channel#12, col_name#13, d_year#6, d_qoy#7, i_category#10, sales_cnt#48, sales_amt#49]
+Arguments: 100, [channel#12 ASC NULLS FIRST, col_name#13 ASC NULLS FIRST, d_year#6 ASC NULLS FIRST, d_qoy#7 ASC NULLS FIRST, i_category#10 ASC NULLS FIRST], [channel#12, col_name#13, d_year#6, d_qoy#7, i_category#10, sales_cnt#48, sales_amt#49]
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/explain.txt
index 9807ba4eccff2..026c9396cd025 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/explain.txt
@@ -208,5 +208,5 @@ Results [7]: [channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6, count(1
(38) TakeOrderedAndProject
Input [7]: [channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6, sales_cnt#46, sales_amt#47]
-Arguments: 100, 0, [channel#12 ASC NULLS FIRST, col_name#13 ASC NULLS FIRST, d_year#9 ASC NULLS FIRST, d_qoy#10 ASC NULLS FIRST, i_category#6 ASC NULLS FIRST], [channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6, sales_cnt#46, sales_amt#47]
+Arguments: 100, [channel#12 ASC NULLS FIRST, col_name#13 ASC NULLS FIRST, d_year#9 ASC NULLS FIRST, d_qoy#10 ASC NULLS FIRST, i_category#6 ASC NULLS FIRST], [channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6, sales_cnt#46, sales_amt#47]
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.sf100/explain.txt
index 84d6f02a16373..0a398a55f7975 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.sf100/explain.txt
@@ -487,7 +487,7 @@ Results [5]: [channel#105, id#106, sum(sales#16)#121 AS sales#124, sum(returns#3
(85) TakeOrderedAndProject
Input [5]: [channel#105, id#106, sales#124, returns#125, profit#126]
-Arguments: 100, 0, [channel#105 ASC NULLS FIRST, id#106 ASC NULLS FIRST], [channel#105, id#106, sales#124, returns#125, profit#126]
+Arguments: 100, [channel#105 ASC NULLS FIRST, id#106 ASC NULLS FIRST], [channel#105, id#106, sales#124, returns#125, profit#126]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/explain.txt
index 86eb206b1415f..894399b2e6061 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/explain.txt
@@ -487,7 +487,7 @@ Results [5]: [channel#105, id#106, sum(sales#16)#121 AS sales#124, sum(returns#3
(85) TakeOrderedAndProject
Input [5]: [channel#105, id#106, sales#124, returns#125, profit#126]
-Arguments: 100, 0, [channel#105 ASC NULLS FIRST, id#106 ASC NULLS FIRST], [channel#105, id#106, sales#124, returns#125, profit#126]
+Arguments: 100, [channel#105 ASC NULLS FIRST, id#106 ASC NULLS FIRST], [channel#105, id#106, sales#124, returns#125, profit#126]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79.sf100/explain.txt
index b188bdb56ba24..386e889f9ef4a 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79.sf100/explain.txt
@@ -182,7 +182,7 @@ Input [8]: [ss_ticket_number#5, ss_customer_sk#1, s_city#17, amt#26, profit#27,
(33) TakeOrderedAndProject
Input [7]: [c_last_name#31, c_first_name#30, substr(s_city, 1, 30)#33, ss_ticket_number#5, amt#26, profit#27, s_city#17]
-Arguments: 100, 0, [c_last_name#31 ASC NULLS FIRST, c_first_name#30 ASC NULLS FIRST, substr(s_city#17, 1, 30) ASC NULLS FIRST, profit#27 ASC NULLS FIRST], [c_last_name#31, c_first_name#30, substr(s_city, 1, 30)#33, ss_ticket_number#5, amt#26, profit#27]
+Arguments: 100, [c_last_name#31 ASC NULLS FIRST, c_first_name#30 ASC NULLS FIRST, substr(s_city#17, 1, 30) ASC NULLS FIRST, profit#27 ASC NULLS FIRST], [c_last_name#31, c_first_name#30, substr(s_city, 1, 30)#33, ss_ticket_number#5, amt#26, profit#27]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79/explain.txt
index 6d1ab2c4b677f..723a46f2bbcf4 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79/explain.txt
@@ -167,7 +167,7 @@ Input [8]: [ss_ticket_number#5, ss_customer_sk#1, s_city#13, amt#26, profit#27,
(30) TakeOrderedAndProject
Input [7]: [c_last_name#30, c_first_name#29, substr(s_city, 1, 30)#32, ss_ticket_number#5, amt#26, profit#27, s_city#13]
-Arguments: 100, 0, [c_last_name#30 ASC NULLS FIRST, c_first_name#29 ASC NULLS FIRST, substr(s_city#13, 1, 30) ASC NULLS FIRST, profit#27 ASC NULLS FIRST], [c_last_name#30, c_first_name#29, substr(s_city, 1, 30)#32, ss_ticket_number#5, amt#26, profit#27]
+Arguments: 100, [c_last_name#30 ASC NULLS FIRST, c_first_name#29 ASC NULLS FIRST, substr(s_city#13, 1, 30) ASC NULLS FIRST, profit#27 ASC NULLS FIRST], [c_last_name#30, c_first_name#29, substr(s_city, 1, 30)#32, ss_ticket_number#5, amt#26, profit#27]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.sf100/explain.txt
index 3d4b5e63923e9..e8c77e7de05eb 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.sf100/explain.txt
@@ -276,7 +276,7 @@ Results [2]: [s_store_name#7, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#31
(49) TakeOrderedAndProject
Input [2]: [s_store_name#7, sum(ss_net_profit)#32]
-Arguments: 100, 0, [s_store_name#7 ASC NULLS FIRST], [s_store_name#7, sum(ss_net_profit)#32]
+Arguments: 100, [s_store_name#7 ASC NULLS FIRST], [s_store_name#7, sum(ss_net_profit)#32]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/explain.txt
index b24e9664aabdc..efac82f31fdb4 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/explain.txt
@@ -246,7 +246,7 @@ Results [2]: [s_store_name#7, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#29
(43) TakeOrderedAndProject
Input [2]: [s_store_name#7, sum(ss_net_profit)#30]
-Arguments: 100, 0, [s_store_name#7 ASC NULLS FIRST], [s_store_name#7, sum(ss_net_profit)#30]
+Arguments: 100, [s_store_name#7 ASC NULLS FIRST], [s_store_name#7, sum(ss_net_profit)#30]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.sf100/explain.txt
index a542ba5b88748..26acaf66589da 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.sf100/explain.txt
@@ -589,7 +589,7 @@ Results [5]: [channel#123, id#124, sum(sales#40)#139 AS sales#142, sum(returns#4
(107) TakeOrderedAndProject
Input [5]: [channel#123, id#124, sales#142, returns#143, profit#144]
-Arguments: 100, 0, [channel#123 ASC NULLS FIRST, id#124 ASC NULLS FIRST], [channel#123, id#124, sales#142, returns#143, profit#144]
+Arguments: 100, [channel#123 ASC NULLS FIRST, id#124 ASC NULLS FIRST], [channel#123, id#124, sales#142, returns#143, profit#144]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/explain.txt
index 53ec7c76b0d54..02df641cf5a20 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/explain.txt
@@ -589,7 +589,7 @@ Results [5]: [channel#123, id#124, sum(sales#40)#139 AS sales#142, sum(returns#4
(107) TakeOrderedAndProject
Input [5]: [channel#123, id#124, sales#142, returns#143, profit#144]
-Arguments: 100, 0, [channel#123 ASC NULLS FIRST, id#124 ASC NULLS FIRST], [channel#123, id#124, sales#142, returns#143, profit#144]
+Arguments: 100, [channel#123 ASC NULLS FIRST, id#124 ASC NULLS FIRST], [channel#123, id#124, sales#142, returns#143, profit#144]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.sf100/explain.txt
index d7c54442bccb6..304c487dab724 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.sf100/explain.txt
@@ -318,7 +318,7 @@ Input [19]: [c_customer_id#2, c_salutation#4, c_first_name#5, c_last_name#6, ca_
(57) TakeOrderedAndProject
Input [16]: [c_customer_id#2, c_salutation#4, c_first_name#5, c_last_name#6, ca_street_number#8, ca_street_name#9, ca_street_type#10, ca_suite_number#11, ca_city#12, ca_county#13, ca_state#14, ca_zip#15, ca_country#16, ca_gmt_offset#17, ca_location_type#18, ctr_total_return#37]
-Arguments: 100, 0, [c_customer_id#2 ASC NULLS FIRST, c_salutation#4 ASC NULLS FIRST, c_first_name#5 ASC NULLS FIRST, c_last_name#6 ASC NULLS FIRST, ca_street_number#8 ASC NULLS FIRST, ca_street_name#9 ASC NULLS FIRST, ca_street_type#10 ASC NULLS FIRST, ca_suite_number#11 ASC NULLS FIRST, ca_city#12 ASC NULLS FIRST, ca_county#13 ASC NULLS FIRST, ca_state#14 ASC NULLS FIRST, ca_zip#15 ASC NULLS FIRST, ca_country#16 ASC NULLS FIRST, ca_gmt_offset#17 ASC NULLS FIRST, ca_location_type#18 ASC NULLS FIRST, ctr_total_return#37 ASC NULLS FIRST], [c_customer_id#2, c_salutation#4, c_first_name#5, c_last_name#6, ca_street_number#8, ca_street_name#9, ca_street_type#10, ca_suite_number#11, ca_city#12, ca_county#13, ca_state#14, ca_zip#15, ca_country#16, ca_gmt_offset#17, ca_location_type#18, ctr_total_return#37]
+Arguments: 100, [c_customer_id#2 ASC NULLS FIRST, c_salutation#4 ASC NULLS FIRST, c_first_name#5 ASC NULLS FIRST, c_last_name#6 ASC NULLS FIRST, ca_street_number#8 ASC NULLS FIRST, ca_street_name#9 ASC NULLS FIRST, ca_street_type#10 ASC NULLS FIRST, ca_suite_number#11 ASC NULLS FIRST, ca_city#12 ASC NULLS FIRST, ca_county#13 ASC NULLS FIRST, ca_state#14 ASC NULLS FIRST, ca_zip#15 ASC NULLS FIRST, ca_country#16 ASC NULLS FIRST, ca_gmt_offset#17 ASC NULLS FIRST, ca_location_type#18 ASC NULLS FIRST, ctr_total_return#37 ASC NULLS FIRST], [c_customer_id#2, c_salutation#4, c_first_name#5, c_last_name#6, ca_street_number#8, ca_street_name#9, ca_street_type#10, ca_suite_number#11, ca_city#12, ca_county#13, ca_state#14, ca_zip#15, ca_country#16, ca_gmt_offset#17, ca_location_type#18, ctr_total_return#37]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/explain.txt
index f2a8ee58cd694..21992cf79a0ba 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/explain.txt
@@ -273,7 +273,7 @@ Input [18]: [ctr_total_return#16, c_customer_id#30, c_current_addr_sk#31, c_salu
(48) TakeOrderedAndProject
Input [16]: [c_customer_id#30, c_salutation#32, c_first_name#33, c_last_name#34, ca_street_number#37, ca_street_name#38, ca_street_type#39, ca_suite_number#40, ca_city#41, ca_county#42, ca_state#43, ca_zip#44, ca_country#45, ca_gmt_offset#46, ca_location_type#47, ctr_total_return#16]
-Arguments: 100, 0, [c_customer_id#30 ASC NULLS FIRST, c_salutation#32 ASC NULLS FIRST, c_first_name#33 ASC NULLS FIRST, c_last_name#34 ASC NULLS FIRST, ca_street_number#37 ASC NULLS FIRST, ca_street_name#38 ASC NULLS FIRST, ca_street_type#39 ASC NULLS FIRST, ca_suite_number#40 ASC NULLS FIRST, ca_city#41 ASC NULLS FIRST, ca_county#42 ASC NULLS FIRST, ca_state#43 ASC NULLS FIRST, ca_zip#44 ASC NULLS FIRST, ca_country#45 ASC NULLS FIRST, ca_gmt_offset#46 ASC NULLS FIRST, ca_location_type#47 ASC NULLS FIRST, ctr_total_return#16 ASC NULLS FIRST], [c_customer_id#30, c_salutation#32, c_first_name#33, c_last_name#34, ca_street_number#37, ca_street_name#38, ca_street_type#39, ca_suite_number#40, ca_city#41, ca_county#42, ca_state#43, ca_zip#44, ca_country#45, ca_gmt_offset#46, ca_location_type#47, ctr_total_return#16]
+Arguments: 100, [c_customer_id#30 ASC NULLS FIRST, c_salutation#32 ASC NULLS FIRST, c_first_name#33 ASC NULLS FIRST, c_last_name#34 ASC NULLS FIRST, ca_street_number#37 ASC NULLS FIRST, ca_street_name#38 ASC NULLS FIRST, ca_street_type#39 ASC NULLS FIRST, ca_suite_number#40 ASC NULLS FIRST, ca_city#41 ASC NULLS FIRST, ca_county#42 ASC NULLS FIRST, ca_state#43 ASC NULLS FIRST, ca_zip#44 ASC NULLS FIRST, ca_country#45 ASC NULLS FIRST, ca_gmt_offset#46 ASC NULLS FIRST, ca_location_type#47 ASC NULLS FIRST, ctr_total_return#16 ASC NULLS FIRST], [c_customer_id#30, c_salutation#32, c_first_name#33, c_last_name#34, ca_street_number#37, ca_street_name#38, ca_street_type#39, ca_suite_number#40, ca_city#41, ca_county#42, ca_state#43, ca_zip#44, ca_country#45, ca_gmt_offset#46, ca_location_type#47, ctr_total_return#16]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.sf100/explain.txt
index 5228c6f22a16d..1e0c78c7329c8 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.sf100/explain.txt
@@ -154,7 +154,7 @@ Results [3]: [i_item_id#2, i_item_desc#3, i_current_price#4]
(28) TakeOrderedAndProject
Input [3]: [i_item_id#2, i_item_desc#3, i_current_price#4]
-Arguments: 100, 0, [i_item_id#2 ASC NULLS FIRST], [i_item_id#2, i_item_desc#3, i_current_price#4]
+Arguments: 100, [i_item_id#2 ASC NULLS FIRST], [i_item_id#2, i_item_desc#3, i_current_price#4]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/explain.txt
index d9ef291bc6937..4e9ad2199c037 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/explain.txt
@@ -139,7 +139,7 @@ Results [3]: [i_item_id#2, i_item_desc#3, i_current_price#4]
(25) TakeOrderedAndProject
Input [3]: [i_item_id#2, i_item_desc#3, i_current_price#4]
-Arguments: 100, 0, [i_item_id#2 ASC NULLS FIRST], [i_item_id#2, i_item_desc#3, i_current_price#4]
+Arguments: 100, [i_item_id#2 ASC NULLS FIRST], [i_item_id#2, i_item_desc#3, i_current_price#4]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/explain.txt
index 8dbbc0b7ece92..6cac91a51f73a 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/explain.txt
@@ -261,7 +261,7 @@ Input [5]: [item_id#13, sr_item_qty#14, cr_item_qty#26, item_id#38, wr_item_qty#
(46) TakeOrderedAndProject
Input [8]: [item_id#13, sr_item_qty#14, sr_dev#41, cr_item_qty#26, cr_dev#42, wr_item_qty#39, wr_dev#43, average#44]
-Arguments: 100, 0, [item_id#13 ASC NULLS FIRST, sr_item_qty#14 ASC NULLS FIRST], [item_id#13, sr_item_qty#14, sr_dev#41, cr_item_qty#26, cr_dev#42, wr_item_qty#39, wr_dev#43, average#44]
+Arguments: 100, [item_id#13 ASC NULLS FIRST, sr_item_qty#14 ASC NULLS FIRST], [item_id#13, sr_item_qty#14, sr_dev#41, cr_item_qty#26, cr_dev#42, wr_item_qty#39, wr_dev#43, average#44]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/explain.txt
index 19b201c2ff155..7e76d4b527ad8 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/explain.txt
@@ -261,7 +261,7 @@ Input [5]: [item_id#13, sr_item_qty#14, cr_item_qty#26, item_id#38, wr_item_qty#
(46) TakeOrderedAndProject
Input [8]: [item_id#13, sr_item_qty#14, sr_dev#41, cr_item_qty#26, cr_dev#42, wr_item_qty#39, wr_dev#43, average#44]
-Arguments: 100, 0, [item_id#13 ASC NULLS FIRST, sr_item_qty#14 ASC NULLS FIRST], [item_id#13, sr_item_qty#14, sr_dev#41, cr_item_qty#26, cr_dev#42, wr_item_qty#39, wr_dev#43, average#44]
+Arguments: 100, [item_id#13 ASC NULLS FIRST, sr_item_qty#14 ASC NULLS FIRST], [item_id#13, sr_item_qty#14, sr_dev#41, cr_item_qty#26, cr_dev#42, wr_item_qty#39, wr_dev#43, average#44]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84.sf100/explain.txt
index 4991ef5b3f374..9762d51e943e8 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84.sf100/explain.txt
@@ -201,5 +201,5 @@ Input [5]: [c_customer_id#1, c_first_name#5, c_last_name#6, cd_demo_sk#18, sr_cd
(37) TakeOrderedAndProject
Input [3]: [customer_id#22, customername#23, c_customer_id#1]
-Arguments: 100, 0, [c_customer_id#1 ASC NULLS FIRST], [customer_id#22, customername#23]
+Arguments: 100, [c_customer_id#1 ASC NULLS FIRST], [customer_id#22, customername#23]
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84/explain.txt
index baf0664cd7c65..d5b84f52f4b00 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84/explain.txt
@@ -201,5 +201,5 @@ Input [5]: [c_customer_id#1, c_first_name#5, c_last_name#6, cd_demo_sk#10, sr_cd
(37) TakeOrderedAndProject
Input [3]: [customer_id#22, customername#23, c_customer_id#1]
-Arguments: 100, 0, [c_customer_id#1 ASC NULLS FIRST], [customer_id#22, customername#23]
+Arguments: 100, [c_customer_id#1 ASC NULLS FIRST], [customer_id#22, customername#23]
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.sf100/explain.txt
index 31a12b816aee4..82f2b017b6bf5 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.sf100/explain.txt
@@ -296,7 +296,7 @@ Results [4]: [substr(r_reason_desc#37, 1, 20) AS substr(r_reason_desc, 1, 20)#55
(54) TakeOrderedAndProject
Input [4]: [substr(r_reason_desc, 1, 20)#55, avg(ws_quantity)#56, avg(wr_refunded_cash)#57, avg(wr_fee)#58]
-Arguments: 100, 0, [substr(r_reason_desc, 1, 20)#55 ASC NULLS FIRST, avg(ws_quantity)#56 ASC NULLS FIRST, avg(wr_refunded_cash)#57 ASC NULLS FIRST, avg(wr_fee)#58 ASC NULLS FIRST], [substr(r_reason_desc, 1, 20)#55, avg(ws_quantity)#56, avg(wr_refunded_cash)#57, avg(wr_fee)#58]
+Arguments: 100, [substr(r_reason_desc, 1, 20)#55 ASC NULLS FIRST, avg(ws_quantity)#56 ASC NULLS FIRST, avg(wr_refunded_cash)#57 ASC NULLS FIRST, avg(wr_fee)#58 ASC NULLS FIRST], [substr(r_reason_desc, 1, 20)#55, avg(ws_quantity)#56, avg(wr_refunded_cash)#57, avg(wr_fee)#58]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/explain.txt
index 8cf66b3f6213d..6ce464ad0805e 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/explain.txt
@@ -266,7 +266,7 @@ Results [4]: [substr(r_reason_desc#35, 1, 20) AS substr(r_reason_desc, 1, 20)#53
(48) TakeOrderedAndProject
Input [4]: [substr(r_reason_desc, 1, 20)#53, avg(ws_quantity)#54, avg(wr_refunded_cash)#55, avg(wr_fee)#56]
-Arguments: 100, 0, [substr(r_reason_desc, 1, 20)#53 ASC NULLS FIRST, avg(ws_quantity)#54 ASC NULLS FIRST, avg(wr_refunded_cash)#55 ASC NULLS FIRST, avg(wr_fee)#56 ASC NULLS FIRST], [substr(r_reason_desc, 1, 20)#53, avg(ws_quantity)#54, avg(wr_refunded_cash)#55, avg(wr_fee)#56]
+Arguments: 100, [substr(r_reason_desc, 1, 20)#53 ASC NULLS FIRST, avg(ws_quantity)#54 ASC NULLS FIRST, avg(wr_refunded_cash)#55 ASC NULLS FIRST, avg(wr_fee)#56 ASC NULLS FIRST], [substr(r_reason_desc, 1, 20)#53, avg(ws_quantity)#54, avg(wr_refunded_cash)#55, avg(wr_fee)#56]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.sf100/explain.txt
index 76962196c8414..0cc089ebeb840 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.sf100/explain.txt
@@ -116,7 +116,7 @@ Input [8]: [total_sum#17, i_category#10, i_class#11, lochierarchy#18, _w1#19, _w
(21) TakeOrderedAndProject
Input [5]: [total_sum#17, i_category#10, i_class#11, lochierarchy#18, rank_within_parent#23]
-Arguments: 100, 0, [lochierarchy#18 DESC NULLS LAST, CASE WHEN (lochierarchy#18 = 0) THEN i_category#10 END ASC NULLS FIRST, rank_within_parent#23 ASC NULLS FIRST], [total_sum#17, i_category#10, i_class#11, lochierarchy#18, rank_within_parent#23]
+Arguments: 100, [lochierarchy#18 DESC NULLS LAST, CASE WHEN (lochierarchy#18 = 0) THEN i_category#10 END ASC NULLS FIRST, rank_within_parent#23 ASC NULLS FIRST], [total_sum#17, i_category#10, i_class#11, lochierarchy#18, rank_within_parent#23]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/explain.txt
index 76962196c8414..0cc089ebeb840 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/explain.txt
@@ -116,7 +116,7 @@ Input [8]: [total_sum#17, i_category#10, i_class#11, lochierarchy#18, _w1#19, _w
(21) TakeOrderedAndProject
Input [5]: [total_sum#17, i_category#10, i_class#11, lochierarchy#18, rank_within_parent#23]
-Arguments: 100, 0, [lochierarchy#18 DESC NULLS LAST, CASE WHEN (lochierarchy#18 = 0) THEN i_category#10 END ASC NULLS FIRST, rank_within_parent#23 ASC NULLS FIRST], [total_sum#17, i_category#10, i_class#11, lochierarchy#18, rank_within_parent#23]
+Arguments: 100, [lochierarchy#18 DESC NULLS LAST, CASE WHEN (lochierarchy#18 = 0) THEN i_category#10 END ASC NULLS FIRST, rank_within_parent#23 ASC NULLS FIRST], [total_sum#17, i_category#10, i_class#11, lochierarchy#18, rank_within_parent#23]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.sf100/explain.txt
index f2f11063b3b6f..3be73b34cb9f1 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.sf100/explain.txt
@@ -149,7 +149,7 @@ Input [9]: [i_category#4, i_class#3, i_brand#2, s_store_name#14, s_company_name#
(27) TakeOrderedAndProject
Input [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#14, s_company_name#15, d_moy#12, sum_sales#21, avg_monthly_sales#24]
-Arguments: 100, 0, [(sum_sales#21 - avg_monthly_sales#24) ASC NULLS FIRST, s_store_name#14 ASC NULLS FIRST], [i_category#4, i_class#3, i_brand#2, s_store_name#14, s_company_name#15, d_moy#12, sum_sales#21, avg_monthly_sales#24]
+Arguments: 100, [(sum_sales#21 - avg_monthly_sales#24) ASC NULLS FIRST, s_store_name#14 ASC NULLS FIRST], [i_category#4, i_class#3, i_brand#2, s_store_name#14, s_company_name#15, d_moy#12, sum_sales#21, avg_monthly_sales#24]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/explain.txt
index 472c590e5518c..5a572dc71127f 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/explain.txt
@@ -149,7 +149,7 @@ Input [9]: [i_category#4, i_class#3, i_brand#2, s_store_name#14, s_company_name#
(27) TakeOrderedAndProject
Input [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#14, s_company_name#15, d_moy#12, sum_sales#21, avg_monthly_sales#24]
-Arguments: 100, 0, [(sum_sales#21 - avg_monthly_sales#24) ASC NULLS FIRST, s_store_name#14 ASC NULLS FIRST], [i_category#4, i_class#3, i_brand#2, s_store_name#14, s_company_name#15, d_moy#12, sum_sales#21, avg_monthly_sales#24]
+Arguments: 100, [(sum_sales#21 - avg_monthly_sales#24) ASC NULLS FIRST, s_store_name#14 ASC NULLS FIRST], [i_category#4, i_class#3, i_brand#2, s_store_name#14, s_company_name#15, d_moy#12, sum_sales#21, avg_monthly_sales#24]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.sf100/explain.txt
index 07d897082d46c..57cc6bfc9ae23 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.sf100/explain.txt
@@ -132,5 +132,5 @@ Results [2]: [ss_customer_sk#11, sum(act_sales#17)#23 AS sumsales#24]
(24) TakeOrderedAndProject
Input [2]: [ss_customer_sk#11, sumsales#24]
-Arguments: 100, 0, [sumsales#24 ASC NULLS FIRST, ss_customer_sk#11 ASC NULLS FIRST], [ss_customer_sk#11, sumsales#24]
+Arguments: 100, [sumsales#24 ASC NULLS FIRST, ss_customer_sk#11 ASC NULLS FIRST], [ss_customer_sk#11, sumsales#24]
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/explain.txt
index f79a19dec0dbb..06f693cc164c0 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/explain.txt
@@ -132,5 +132,5 @@ Results [2]: [ss_customer_sk#2, sum(act_sales#17)#23 AS sumsales#24]
(24) TakeOrderedAndProject
Input [2]: [ss_customer_sk#2, sumsales#24]
-Arguments: 100, 0, [sumsales#24 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST], [ss_customer_sk#2, sumsales#24]
+Arguments: 100, [sumsales#24 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST], [ss_customer_sk#2, sumsales#24]
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99.sf100/explain.txt
index 5678826cdde44..1b955ee3bd96c 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99.sf100/explain.txt
@@ -179,5 +179,5 @@ Results [8]: [_groupingexpression#18 AS substr(w_warehouse_name, 1, 20)#35, sm_t
(32) TakeOrderedAndProject
Input [8]: [substr(w_warehouse_name, 1, 20)#35, sm_type#10, cc_name#13, 30 days #36, 31 - 60 days #37, 61 - 90 days #38, 91 - 120 days #39, >120 days #40]
-Arguments: 100, 0, [substr(w_warehouse_name, 1, 20)#35 ASC NULLS FIRST, sm_type#10 ASC NULLS FIRST, cc_name#13 ASC NULLS FIRST], [substr(w_warehouse_name, 1, 20)#35, sm_type#10, cc_name#13, 30 days #36, 31 - 60 days #37, 61 - 90 days #38, 91 - 120 days #39, >120 days #40]
+Arguments: 100, [substr(w_warehouse_name, 1, 20)#35 ASC NULLS FIRST, sm_type#10 ASC NULLS FIRST, cc_name#13 ASC NULLS FIRST], [substr(w_warehouse_name, 1, 20)#35, sm_type#10, cc_name#13, 30 days #36, 31 - 60 days #37, 61 - 90 days #38, 91 - 120 days #39, >120 days #40]
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99/explain.txt
index 0c6114d1cbd91..1431623539828 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99/explain.txt
@@ -179,5 +179,5 @@ Results [8]: [_groupingexpression#18 AS substr(w_warehouse_name, 1, 20)#35, sm_t
(32) TakeOrderedAndProject
Input [8]: [substr(w_warehouse_name, 1, 20)#35, sm_type#10, cc_name#13, 30 days #36, 31 - 60 days #37, 61 - 90 days #38, 91 - 120 days #39, >120 days #40]
-Arguments: 100, 0, [substr(w_warehouse_name, 1, 20)#35 ASC NULLS FIRST, sm_type#10 ASC NULLS FIRST, cc_name#13 ASC NULLS FIRST], [substr(w_warehouse_name, 1, 20)#35, sm_type#10, cc_name#13, 30 days #36, 31 - 60 days #37, 61 - 90 days #38, 91 - 120 days #39, >120 days #40]
+Arguments: 100, [substr(w_warehouse_name, 1, 20)#35 ASC NULLS FIRST, sm_type#10 ASC NULLS FIRST, cc_name#13 ASC NULLS FIRST], [substr(w_warehouse_name, 1, 20)#35, sm_type#10, cc_name#13, 30 days #36, 31 - 60 days #37, 61 - 90 days #38, 91 - 120 days #39, >120 days #40]
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.sf100/explain.txt
index ef1337b7389a1..fe97109236cf2 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.sf100/explain.txt
@@ -244,7 +244,7 @@ Results [14]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, count
(45) TakeOrderedAndProject
Input [14]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cnt1#36, cd_purchase_estimate#27, cnt2#37, cd_credit_rating#28, cnt3#38, cd_dep_count#29, cnt4#39, cd_dep_employed_count#30, cnt5#40, cd_dep_college_count#31, cnt6#41]
-Arguments: 100, 0, [cd_gender#24 ASC NULLS FIRST, cd_marital_status#25 ASC NULLS FIRST, cd_education_status#26 ASC NULLS FIRST, cd_purchase_estimate#27 ASC NULLS FIRST, cd_credit_rating#28 ASC NULLS FIRST, cd_dep_count#29 ASC NULLS FIRST, cd_dep_employed_count#30 ASC NULLS FIRST, cd_dep_college_count#31 ASC NULLS FIRST], [cd_gender#24, cd_marital_status#25, cd_education_status#26, cnt1#36, cd_purchase_estimate#27, cnt2#37, cd_credit_rating#28, cnt3#38, cd_dep_count#29, cnt4#39, cd_dep_employed_count#30, cnt5#40, cd_dep_college_count#31, cnt6#41]
+Arguments: 100, [cd_gender#24 ASC NULLS FIRST, cd_marital_status#25 ASC NULLS FIRST, cd_education_status#26 ASC NULLS FIRST, cd_purchase_estimate#27 ASC NULLS FIRST, cd_credit_rating#28 ASC NULLS FIRST, cd_dep_count#29 ASC NULLS FIRST, cd_dep_employed_count#30 ASC NULLS FIRST, cd_dep_college_count#31 ASC NULLS FIRST], [cd_gender#24, cd_marital_status#25, cd_education_status#26, cnt1#36, cd_purchase_estimate#27, cnt2#37, cd_credit_rating#28, cnt3#38, cd_dep_count#29, cnt4#39, cd_dep_employed_count#30, cnt5#40, cd_dep_college_count#31, cnt6#41]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/explain.txt
index 2c598e386e1b0..02522cd257d71 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/explain.txt
@@ -224,7 +224,7 @@ Results [14]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, count
(41) TakeOrderedAndProject
Input [14]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, cnt1#35, cd_purchase_estimate#25, cnt2#36, cd_credit_rating#26, cnt3#37, cd_dep_count#27, cnt4#38, cd_dep_employed_count#28, cnt5#39, cd_dep_college_count#29, cnt6#40]
-Arguments: 100, 0, [cd_gender#22 ASC NULLS FIRST, cd_marital_status#23 ASC NULLS FIRST, cd_education_status#24 ASC NULLS FIRST, cd_purchase_estimate#25 ASC NULLS FIRST, cd_credit_rating#26 ASC NULLS FIRST, cd_dep_count#27 ASC NULLS FIRST, cd_dep_employed_count#28 ASC NULLS FIRST, cd_dep_college_count#29 ASC NULLS FIRST], [cd_gender#22, cd_marital_status#23, cd_education_status#24, cnt1#35, cd_purchase_estimate#25, cnt2#36, cd_credit_rating#26, cnt3#37, cd_dep_count#27, cnt4#38, cd_dep_employed_count#28, cnt5#39, cd_dep_college_count#29, cnt6#40]
+Arguments: 100, [cd_gender#22 ASC NULLS FIRST, cd_marital_status#23 ASC NULLS FIRST, cd_education_status#24 ASC NULLS FIRST, cd_purchase_estimate#25 ASC NULLS FIRST, cd_credit_rating#26 ASC NULLS FIRST, cd_dep_count#27 ASC NULLS FIRST, cd_dep_employed_count#28 ASC NULLS FIRST, cd_dep_college_count#29 ASC NULLS FIRST], [cd_gender#22, cd_marital_status#23, cd_education_status#24, cnt1#35, cd_purchase_estimate#25, cnt2#36, cd_credit_rating#26, cnt3#37, cd_dep_count#27, cnt4#38, cd_dep_employed_count#28, cnt5#39, cd_dep_college_count#29, cnt6#40]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.sf100/explain.txt
index 76fa78e508d06..a01e3ea970a41 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.sf100/explain.txt
@@ -441,7 +441,7 @@ Input [10]: [customer_id#22, year_total#23, customer_id#44, customer_first_name#
(80) TakeOrderedAndProject
Input [4]: [customer_id#44, customer_first_name#45, customer_last_name#46, customer_email_address#47]
-Arguments: 100, 0, [customer_id#44 ASC NULLS FIRST, customer_first_name#45 ASC NULLS FIRST, customer_last_name#46 ASC NULLS FIRST, customer_email_address#47 ASC NULLS FIRST], [customer_id#44, customer_first_name#45, customer_last_name#46, customer_email_address#47]
+Arguments: 100, [customer_id#44 ASC NULLS FIRST, customer_first_name#45 ASC NULLS FIRST, customer_last_name#46 ASC NULLS FIRST, customer_email_address#47 ASC NULLS FIRST], [customer_id#44, customer_first_name#45, customer_last_name#46, customer_email_address#47]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/explain.txt
index 6b45c59713742..3b77e772600e0 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/explain.txt
@@ -410,7 +410,7 @@ Input [10]: [customer_id#21, year_total#22, customer_id#42, customer_first_name#
(72) TakeOrderedAndProject
Input [4]: [customer_id#42, customer_first_name#43, customer_last_name#44, customer_email_address#45]
-Arguments: 100, 0, [customer_id#42 ASC NULLS FIRST, customer_first_name#43 ASC NULLS FIRST, customer_last_name#44 ASC NULLS FIRST, customer_email_address#45 ASC NULLS FIRST], [customer_id#42, customer_first_name#43, customer_last_name#44, customer_email_address#45]
+Arguments: 100, [customer_id#42 ASC NULLS FIRST, customer_first_name#43 ASC NULLS FIRST, customer_last_name#44 ASC NULLS FIRST, customer_email_address#45 ASC NULLS FIRST], [customer_id#42, customer_first_name#43, customer_last_name#44, customer_email_address#45]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.sf100/explain.txt
index b011014d38f09..8cdf63ac2c420 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.sf100/explain.txt
@@ -126,7 +126,7 @@ Input [9]: [i_item_id#7, i_item_desc#8, i_category#11, i_class#10, i_current_pri
(23) TakeOrderedAndProject
Input [7]: [i_item_id#7, i_item_desc#8, i_category#11, i_class#10, i_current_price#9, itemrevenue#18, revenueratio#23]
-Arguments: 100, 0, [i_category#11 ASC NULLS FIRST, i_class#10 ASC NULLS FIRST, i_item_id#7 ASC NULLS FIRST, i_item_desc#8 ASC NULLS FIRST, revenueratio#23 ASC NULLS FIRST], [i_item_id#7, i_item_desc#8, i_category#11, i_class#10, i_current_price#9, itemrevenue#18, revenueratio#23]
+Arguments: 100, [i_category#11 ASC NULLS FIRST, i_class#10 ASC NULLS FIRST, i_item_id#7 ASC NULLS FIRST, i_item_desc#8 ASC NULLS FIRST, revenueratio#23 ASC NULLS FIRST], [i_item_id#7, i_item_desc#8, i_category#11, i_class#10, i_current_price#9, itemrevenue#18, revenueratio#23]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/explain.txt
index 7f0cad457a44c..4eac76d9fc5c1 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/explain.txt
@@ -111,7 +111,7 @@ Input [9]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_pric
(20) TakeOrderedAndProject
Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#17, revenueratio#22]
-Arguments: 100, 0, [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#22 ASC NULLS FIRST], [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#17, revenueratio#22]
+Arguments: 100, [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#22 ASC NULLS FIRST], [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#17, revenueratio#22]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/explain.txt
index b983676a12f24..25791c2f6ec4d 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/explain.txt
@@ -564,7 +564,7 @@ Join condition: None
(102) TakeOrderedAndProject
Input [12]: [channel#63, i_brand_id#49, i_class_id#50, i_category_id#51, sales#64, number_sales#65, channel#88, i_brand_id#76, i_class_id#77, i_category_id#78, sales#89, number_sales#90]
-Arguments: 100, 0, [i_brand_id#49 ASC NULLS FIRST, i_class_id#50 ASC NULLS FIRST, i_category_id#51 ASC NULLS FIRST], [channel#63, i_brand_id#49, i_class_id#50, i_category_id#51, sales#64, number_sales#65, channel#88, i_brand_id#76, i_class_id#77, i_category_id#78, sales#89, number_sales#90]
+Arguments: 100, [i_brand_id#49 ASC NULLS FIRST, i_class_id#50 ASC NULLS FIRST, i_category_id#51 ASC NULLS FIRST], [channel#63, i_brand_id#49, i_class_id#50, i_category_id#51, sales#64, number_sales#65, channel#88, i_brand_id#76, i_class_id#77, i_category_id#78, sales#89, number_sales#90]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt
index 385c4861d3a53..aaf1bedf681b8 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt
@@ -484,7 +484,7 @@ Join condition: None
(86) TakeOrderedAndProject
Input [12]: [channel#58, i_brand_id#44, i_class_id#45, i_category_id#46, sales#59, number_sales#60, channel#82, i_brand_id#69, i_class_id#70, i_category_id#71, sales#83, number_sales#84]
-Arguments: 100, 0, [i_brand_id#44 ASC NULLS FIRST, i_class_id#45 ASC NULLS FIRST, i_category_id#46 ASC NULLS FIRST], [channel#58, i_brand_id#44, i_class_id#45, i_category_id#46, sales#59, number_sales#60, channel#82, i_brand_id#69, i_class_id#70, i_category_id#71, sales#83, number_sales#84]
+Arguments: 100, [i_brand_id#44 ASC NULLS FIRST, i_class_id#45 ASC NULLS FIRST, i_category_id#46 ASC NULLS FIRST], [channel#58, i_brand_id#44, i_class_id#45, i_category_id#46, sales#59, number_sales#60, channel#82, i_brand_id#69, i_class_id#70, i_category_id#71, sales#83, number_sales#84]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/explain.txt
index cddfda97aeb20..daa4bcfdfc264 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/explain.txt
@@ -832,7 +832,7 @@ Results [6]: [channel#63, i_brand_id#49, i_class_id#50, i_category_id#51, sum_sa
(146) TakeOrderedAndProject
Input [6]: [channel#63, i_brand_id#49, i_class_id#50, i_category_id#51, sum_sales#121, number_sales#122]
-Arguments: 100, 0, [channel#63 ASC NULLS FIRST, i_brand_id#49 ASC NULLS FIRST, i_class_id#50 ASC NULLS FIRST, i_category_id#51 ASC NULLS FIRST], [channel#63, i_brand_id#49, i_class_id#50, i_category_id#51, sum_sales#121, number_sales#122]
+Arguments: 100, [channel#63 ASC NULLS FIRST, i_brand_id#49 ASC NULLS FIRST, i_class_id#50 ASC NULLS FIRST, i_category_id#51 ASC NULLS FIRST], [channel#63, i_brand_id#49, i_class_id#50, i_category_id#51, sum_sales#121, number_sales#122]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/explain.txt
index 303aab65a4ab9..171318a3df7db 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/explain.txt
@@ -737,7 +737,7 @@ Results [6]: [channel#58, i_brand_id#44, i_class_id#45, i_category_id#46, sum_sa
(127) TakeOrderedAndProject
Input [6]: [channel#58, i_brand_id#44, i_class_id#45, i_category_id#46, sum_sales#114, number_sales#115]
-Arguments: 100, 0, [channel#58 ASC NULLS FIRST, i_brand_id#44 ASC NULLS FIRST, i_class_id#45 ASC NULLS FIRST, i_category_id#46 ASC NULLS FIRST], [channel#58, i_brand_id#44, i_class_id#45, i_category_id#46, sum_sales#114, number_sales#115]
+Arguments: 100, [channel#58 ASC NULLS FIRST, i_brand_id#44 ASC NULLS FIRST, i_class_id#45 ASC NULLS FIRST, i_category_id#46 ASC NULLS FIRST], [channel#58, i_brand_id#44, i_class_id#45, i_category_id#46, sum_sales#114, number_sales#115]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.sf100/explain.txt
index 012bbaa8e45c3..506e18eabcc20 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.sf100/explain.txt
@@ -853,7 +853,7 @@ Results [11]: [null AS i_item_id#266, null AS ca_country#267, null AS ca_state#2
(156) TakeOrderedAndProject
Input [11]: [i_item_id#18, ca_country#29, ca_state#28, ca_county#27, agg1#78, agg2#79, agg3#80, agg4#81, agg5#82, agg6#83, agg7#84]
-Arguments: 100, 0, [ca_country#29 ASC NULLS FIRST, ca_state#28 ASC NULLS FIRST, ca_county#27 ASC NULLS FIRST, i_item_id#18 ASC NULLS FIRST], [i_item_id#18, ca_country#29, ca_state#28, ca_county#27, agg1#78, agg2#79, agg3#80, agg4#81, agg5#82, agg6#83, agg7#84]
+Arguments: 100, [ca_country#29 ASC NULLS FIRST, ca_state#28 ASC NULLS FIRST, ca_county#27 ASC NULLS FIRST, i_item_id#18 ASC NULLS FIRST], [i_item_id#18, ca_country#29, ca_state#28, ca_county#27, agg1#78, agg2#79, agg3#80, agg4#81, agg5#82, agg6#83, agg7#84]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a/explain.txt
index fdfa28cf8ee18..7bd7a7ae80176 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a/explain.txt
@@ -834,7 +834,7 @@ Results [11]: [null AS i_item_id#258, null AS ca_country#259, null AS ca_state#2
(153) TakeOrderedAndProject
Input [11]: [i_item_id#31, ca_country#27, ca_state#26, ca_county#25, agg1#76, agg2#77, agg3#78, agg4#79, agg5#80, agg6#81, agg7#82]
-Arguments: 100, 0, [ca_country#27 ASC NULLS FIRST, ca_state#26 ASC NULLS FIRST, ca_county#25 ASC NULLS FIRST, i_item_id#31 ASC NULLS FIRST], [i_item_id#31, ca_country#27, ca_state#26, ca_county#25, agg1#76, agg2#77, agg3#78, agg4#79, agg5#80, agg6#81, agg7#82]
+Arguments: 100, [ca_country#27 ASC NULLS FIRST, ca_state#26 ASC NULLS FIRST, ca_county#25 ASC NULLS FIRST, i_item_id#31 ASC NULLS FIRST], [i_item_id#31, ca_country#27, ca_state#26, ca_county#25, agg1#76, agg2#77, agg3#78, agg4#79, agg5#80, agg6#81, agg7#82]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.sf100/explain.txt
index 8f10eecb293fb..7306778e34b12 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.sf100/explain.txt
@@ -126,7 +126,7 @@ Input [9]: [i_item_id#7, i_item_desc#8, i_category#11, i_class#10, i_current_pri
(23) TakeOrderedAndProject
Input [7]: [i_item_id#7, i_item_desc#8, i_category#11, i_class#10, i_current_price#9, itemrevenue#18, revenueratio#23]
-Arguments: 100, 0, [i_category#11 ASC NULLS FIRST, i_class#10 ASC NULLS FIRST, i_item_id#7 ASC NULLS FIRST, i_item_desc#8 ASC NULLS FIRST, revenueratio#23 ASC NULLS FIRST], [i_item_id#7, i_item_desc#8, i_category#11, i_class#10, i_current_price#9, itemrevenue#18, revenueratio#23]
+Arguments: 100, [i_category#11 ASC NULLS FIRST, i_class#10 ASC NULLS FIRST, i_item_id#7 ASC NULLS FIRST, i_item_desc#8 ASC NULLS FIRST, revenueratio#23 ASC NULLS FIRST], [i_item_id#7, i_item_desc#8, i_category#11, i_class#10, i_current_price#9, itemrevenue#18, revenueratio#23]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/explain.txt
index 3a1cae03eca2c..e392ff69cffe5 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/explain.txt
@@ -111,7 +111,7 @@ Input [9]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_pric
(20) TakeOrderedAndProject
Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#17, revenueratio#22]
-Arguments: 100, 0, [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#22 ASC NULLS FIRST], [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#17, revenueratio#22]
+Arguments: 100, [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#22 ASC NULLS FIRST], [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#17, revenueratio#22]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22.sf100/explain.txt
index fdd92d584c472..eb956b9e75ab0 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22.sf100/explain.txt
@@ -136,7 +136,7 @@ Results [5]: [i_product_name#14, i_brand#15, i_class#16, i_category#17, avg(inv_
(25) TakeOrderedAndProject
Input [5]: [i_product_name#14, i_brand#15, i_class#16, i_category#17, qoh#25]
-Arguments: 100, 0, [qoh#25 ASC NULLS FIRST, i_product_name#14 ASC NULLS FIRST, i_brand#15 ASC NULLS FIRST, i_class#16 ASC NULLS FIRST, i_category#17 ASC NULLS FIRST], [i_product_name#14, i_brand#15, i_class#16, i_category#17, qoh#25]
+Arguments: 100, [qoh#25 ASC NULLS FIRST, i_product_name#14 ASC NULLS FIRST, i_brand#15 ASC NULLS FIRST, i_class#16 ASC NULLS FIRST, i_category#17 ASC NULLS FIRST], [i_product_name#14, i_brand#15, i_class#16, i_category#17, qoh#25]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/explain.txt
index f912a014c28f8..85c21eca87544 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/explain.txt
@@ -121,7 +121,7 @@ Results [5]: [i_product_name#13, i_brand#14, i_class#15, i_category#16, avg(inv_
(22) TakeOrderedAndProject
Input [5]: [i_product_name#13, i_brand#14, i_class#15, i_category#16, qoh#24]
-Arguments: 100, 0, [qoh#24 ASC NULLS FIRST, i_product_name#13 ASC NULLS FIRST, i_brand#14 ASC NULLS FIRST, i_class#15 ASC NULLS FIRST, i_category#16 ASC NULLS FIRST], [i_product_name#13, i_brand#14, i_class#15, i_category#16, qoh#24]
+Arguments: 100, [qoh#24 ASC NULLS FIRST, i_product_name#13 ASC NULLS FIRST, i_brand#14 ASC NULLS FIRST, i_class#15 ASC NULLS FIRST, i_category#16 ASC NULLS FIRST], [i_product_name#13, i_brand#14, i_class#15, i_category#16, qoh#24]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.sf100/explain.txt
index 58f4ab1df1e4a..e7072101f8f23 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.sf100/explain.txt
@@ -290,7 +290,7 @@ Results [5]: [null AS i_product_name#62, null AS i_brand#63, null AS i_class#64,
(48) TakeOrderedAndProject
Input [5]: [i_product_name#14, i_brand#11, i_class#12, i_category#13, qoh#28]
-Arguments: 100, 0, [qoh#28 ASC NULLS FIRST, i_product_name#14 ASC NULLS FIRST, i_brand#11 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_category#13 ASC NULLS FIRST], [i_product_name#14, i_brand#11, i_class#12, i_category#13, qoh#28]
+Arguments: 100, [qoh#28 ASC NULLS FIRST, i_product_name#14 ASC NULLS FIRST, i_brand#11 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_category#13 ASC NULLS FIRST], [i_product_name#14, i_brand#11, i_class#12, i_category#13, qoh#28]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/explain.txt
index d5d267ed52c3c..c5bef0d13db91 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/explain.txt
@@ -275,7 +275,7 @@ Results [5]: [null AS i_product_name#61, null AS i_brand#62, null AS i_class#63,
(45) TakeOrderedAndProject
Input [5]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, qoh#27]
-Arguments: 100, 0, [qoh#27 ASC NULLS FIRST, i_product_name#11 ASC NULLS FIRST, i_brand#8 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_category#10 ASC NULLS FIRST], [i_product_name#11, i_brand#8, i_class#9, i_category#10, qoh#27]
+Arguments: 100, [qoh#27 ASC NULLS FIRST, i_product_name#11 ASC NULLS FIRST, i_brand#8 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_category#10 ASC NULLS FIRST], [i_product_name#11, i_brand#8, i_class#9, i_category#10, qoh#27]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.sf100/explain.txt
index 69d7b8972b7cc..b55e5641a679a 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.sf100/explain.txt
@@ -404,7 +404,7 @@ Results [7]: [null AS i_item_id#102, null AS s_state#103, 1 AS g_state#104, avg(
(73) TakeOrderedAndProject
Input [7]: [i_item_id#20, s_state#17, g_state#47, agg1#48, agg2#49, agg3#50, agg4#51]
-Arguments: 100, 0, [i_item_id#20 ASC NULLS FIRST, s_state#17 ASC NULLS FIRST], [i_item_id#20, s_state#17, g_state#47, agg1#48, agg2#49, agg3#50, agg4#51]
+Arguments: 100, [i_item_id#20 ASC NULLS FIRST, s_state#17 ASC NULLS FIRST], [i_item_id#20, s_state#17, g_state#47, agg1#48, agg2#49, agg3#50, agg4#51]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a/explain.txt
index 515003801461b..60b1498c4e6d0 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a/explain.txt
@@ -404,7 +404,7 @@ Results [7]: [null AS i_item_id#102, null AS s_state#103, 1 AS g_state#104, avg(
(73) TakeOrderedAndProject
Input [7]: [i_item_id#20, s_state#17, g_state#47, agg1#48, agg2#49, agg3#50, agg4#51]
-Arguments: 100, 0, [i_item_id#20 ASC NULLS FIRST, s_state#17 ASC NULLS FIRST], [i_item_id#20, s_state#17, g_state#47, agg1#48, agg2#49, agg3#50, agg4#51]
+Arguments: 100, [i_item_id#20 ASC NULLS FIRST, s_state#17 ASC NULLS FIRST], [i_item_id#20, s_state#17, g_state#47, agg1#48, agg2#49, agg3#50, agg4#51]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.sf100/explain.txt
index 55eea4d3b5c4b..ba6ce3011207d 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.sf100/explain.txt
@@ -287,7 +287,7 @@ Results [18]: [ca_state#22, cd_gender#26, cd_marital_status#27, cd_dep_count#28,
(53) TakeOrderedAndProject
Input [18]: [ca_state#22, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cnt1#69, avg(cd_dep_count)#70, max(cd_dep_count)#71, sum(cd_dep_count)#72, cd_dep_employed_count#29, cnt2#73, avg(cd_dep_employed_count)#74, max(cd_dep_employed_count)#75, sum(cd_dep_employed_count)#76, cd_dep_college_count#30, cnt3#77, avg(cd_dep_college_count)#78, max(cd_dep_college_count)#79, sum(cd_dep_college_count)#80]
-Arguments: 100, 0, [ca_state#22 ASC NULLS FIRST, cd_gender#26 ASC NULLS FIRST, cd_marital_status#27 ASC NULLS FIRST, cd_dep_count#28 ASC NULLS FIRST, cd_dep_employed_count#29 ASC NULLS FIRST, cd_dep_college_count#30 ASC NULLS FIRST], [ca_state#22, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cnt1#69, avg(cd_dep_count)#70, max(cd_dep_count)#71, sum(cd_dep_count)#72, cd_dep_employed_count#29, cnt2#73, avg(cd_dep_employed_count)#74, max(cd_dep_employed_count)#75, sum(cd_dep_employed_count)#76, cd_dep_college_count#30, cnt3#77, avg(cd_dep_college_count)#78, max(cd_dep_college_count)#79, sum(cd_dep_college_count)#80]
+Arguments: 100, [ca_state#22 ASC NULLS FIRST, cd_gender#26 ASC NULLS FIRST, cd_marital_status#27 ASC NULLS FIRST, cd_dep_count#28 ASC NULLS FIRST, cd_dep_employed_count#29 ASC NULLS FIRST, cd_dep_college_count#30 ASC NULLS FIRST], [ca_state#22, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cnt1#69, avg(cd_dep_count)#70, max(cd_dep_count)#71, sum(cd_dep_count)#72, cd_dep_employed_count#29, cnt2#73, avg(cd_dep_employed_count)#74, max(cd_dep_employed_count)#75, sum(cd_dep_employed_count)#76, cd_dep_college_count#30, cnt3#77, avg(cd_dep_college_count)#78, max(cd_dep_college_count)#79, sum(cd_dep_college_count)#80]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/explain.txt
index c383409e22b74..a7d47450807a7 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/explain.txt
@@ -232,7 +232,7 @@ Results [18]: [ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25,
(42) TakeOrderedAndProject
Input [18]: [ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cnt1#66, avg(cd_dep_count)#67, max(cd_dep_count)#68, sum(cd_dep_count)#69, cd_dep_employed_count#26, cnt2#70, avg(cd_dep_employed_count)#71, max(cd_dep_employed_count)#72, sum(cd_dep_employed_count)#73, cd_dep_college_count#27, cnt3#74, avg(cd_dep_college_count)#75, max(cd_dep_college_count)#76, sum(cd_dep_college_count)#77]
-Arguments: 100, 0, [ca_state#20 ASC NULLS FIRST, cd_gender#23 ASC NULLS FIRST, cd_marital_status#24 ASC NULLS FIRST, cd_dep_count#25 ASC NULLS FIRST, cd_dep_employed_count#26 ASC NULLS FIRST, cd_dep_college_count#27 ASC NULLS FIRST], [ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cnt1#66, avg(cd_dep_count)#67, max(cd_dep_count)#68, sum(cd_dep_count)#69, cd_dep_employed_count#26, cnt2#70, avg(cd_dep_employed_count)#71, max(cd_dep_employed_count)#72, sum(cd_dep_employed_count)#73, cd_dep_college_count#27, cnt3#74, avg(cd_dep_college_count)#75, max(cd_dep_college_count)#76, sum(cd_dep_college_count)#77]
+Arguments: 100, [ca_state#20 ASC NULLS FIRST, cd_gender#23 ASC NULLS FIRST, cd_marital_status#24 ASC NULLS FIRST, cd_dep_count#25 ASC NULLS FIRST, cd_dep_employed_count#26 ASC NULLS FIRST, cd_dep_college_count#27 ASC NULLS FIRST], [ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cnt1#66, avg(cd_dep_count)#67, max(cd_dep_count)#68, sum(cd_dep_count)#69, cd_dep_employed_count#26, cnt2#70, avg(cd_dep_employed_count)#71, max(cd_dep_employed_count)#72, sum(cd_dep_employed_count)#73, cd_dep_college_count#27, cnt3#74, avg(cd_dep_college_count)#75, max(cd_dep_college_count)#76, sum(cd_dep_college_count)#77]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.sf100/explain.txt
index 9bbb67475730a..f8d8946e2af92 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.sf100/explain.txt
@@ -269,7 +269,7 @@ Results [18]: [ca_state#21, cd_gender#25, cd_marital_status#26, cd_dep_count#27,
(50) TakeOrderedAndProject
Input [18]: [ca_state#21, cd_gender#25, cd_marital_status#26, cd_dep_count#27, cnt1#68, avg(cd_dep_count)#69, max(cd_dep_count)#70, sum(cd_dep_count)#71, cd_dep_employed_count#28, cnt2#72, avg(cd_dep_employed_count)#73, max(cd_dep_employed_count)#74, sum(cd_dep_employed_count)#75, cd_dep_college_count#29, cnt3#76, avg(cd_dep_college_count)#77, max(cd_dep_college_count)#78, sum(cd_dep_college_count)#79]
-Arguments: 100, 0, [ca_state#21 ASC NULLS FIRST, cd_gender#25 ASC NULLS FIRST, cd_marital_status#26 ASC NULLS FIRST, cd_dep_count#27 ASC NULLS FIRST, cd_dep_employed_count#28 ASC NULLS FIRST, cd_dep_college_count#29 ASC NULLS FIRST], [ca_state#21, cd_gender#25, cd_marital_status#26, cd_dep_count#27, cnt1#68, avg(cd_dep_count)#69, max(cd_dep_count)#70, sum(cd_dep_count)#71, cd_dep_employed_count#28, cnt2#72, avg(cd_dep_employed_count)#73, max(cd_dep_employed_count)#74, sum(cd_dep_employed_count)#75, cd_dep_college_count#29, cnt3#76, avg(cd_dep_college_count)#77, max(cd_dep_college_count)#78, sum(cd_dep_college_count)#79]
+Arguments: 100, [ca_state#21 ASC NULLS FIRST, cd_gender#25 ASC NULLS FIRST, cd_marital_status#26 ASC NULLS FIRST, cd_dep_count#27 ASC NULLS FIRST, cd_dep_employed_count#28 ASC NULLS FIRST, cd_dep_college_count#29 ASC NULLS FIRST], [ca_state#21, cd_gender#25, cd_marital_status#26, cd_dep_count#27, cnt1#68, avg(cd_dep_count)#69, max(cd_dep_count)#70, sum(cd_dep_count)#71, cd_dep_employed_count#28, cnt2#72, avg(cd_dep_employed_count)#73, max(cd_dep_employed_count)#74, sum(cd_dep_employed_count)#75, cd_dep_college_count#29, cnt3#76, avg(cd_dep_college_count)#77, max(cd_dep_college_count)#78, sum(cd_dep_college_count)#79]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/explain.txt
index 3119516f9d6ff..fe6d15a3fb15b 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/explain.txt
@@ -219,7 +219,7 @@ Results [18]: [ca_state#19, cd_gender#22, cd_marital_status#23, cd_dep_count#24,
(40) TakeOrderedAndProject
Input [18]: [ca_state#19, cd_gender#22, cd_marital_status#23, cd_dep_count#24, cnt1#65, avg(cd_dep_count)#66, max(cd_dep_count)#67, sum(cd_dep_count)#68, cd_dep_employed_count#25, cnt2#69, avg(cd_dep_employed_count)#70, max(cd_dep_employed_count)#71, sum(cd_dep_employed_count)#72, cd_dep_college_count#26, cnt3#73, avg(cd_dep_college_count)#74, max(cd_dep_college_count)#75, sum(cd_dep_college_count)#76]
-Arguments: 100, 0, [ca_state#19 ASC NULLS FIRST, cd_gender#22 ASC NULLS FIRST, cd_marital_status#23 ASC NULLS FIRST, cd_dep_count#24 ASC NULLS FIRST, cd_dep_employed_count#25 ASC NULLS FIRST, cd_dep_college_count#26 ASC NULLS FIRST], [ca_state#19, cd_gender#22, cd_marital_status#23, cd_dep_count#24, cnt1#65, avg(cd_dep_count)#66, max(cd_dep_count)#67, sum(cd_dep_count)#68, cd_dep_employed_count#25, cnt2#69, avg(cd_dep_employed_count)#70, max(cd_dep_employed_count)#71, sum(cd_dep_employed_count)#72, cd_dep_college_count#26, cnt3#73, avg(cd_dep_college_count)#74, max(cd_dep_college_count)#75, sum(cd_dep_college_count)#76]
+Arguments: 100, [ca_state#19 ASC NULLS FIRST, cd_gender#22 ASC NULLS FIRST, cd_marital_status#23 ASC NULLS FIRST, cd_dep_count#24 ASC NULLS FIRST, cd_dep_employed_count#25 ASC NULLS FIRST, cd_dep_college_count#26 ASC NULLS FIRST], [ca_state#19, cd_gender#22, cd_marital_status#23, cd_dep_count#24, cnt1#65, avg(cd_dep_count)#66, max(cd_dep_count)#67, sum(cd_dep_count)#68, cd_dep_employed_count#25, cnt2#69, avg(cd_dep_employed_count)#70, max(cd_dep_employed_count)#71, sum(cd_dep_employed_count)#72, cd_dep_college_count#26, cnt3#73, avg(cd_dep_college_count)#74, max(cd_dep_college_count)#75, sum(cd_dep_college_count)#76]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.sf100/explain.txt
index f93a8221a587e..6302a3b221f62 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.sf100/explain.txt
@@ -239,7 +239,7 @@ Input [6]: [gross_margin#22, i_category#13, i_class#12, lochierarchy#25, _w0#68,
(41) TakeOrderedAndProject
Input [5]: [gross_margin#22, i_category#13, i_class#12, lochierarchy#25, rank_within_parent#70]
-Arguments: 100, 0, [lochierarchy#25 DESC NULLS LAST, CASE WHEN (lochierarchy#25 = 0) THEN i_category#13 END ASC NULLS FIRST, rank_within_parent#70 ASC NULLS FIRST], [gross_margin#22, i_category#13, i_class#12, lochierarchy#25, rank_within_parent#70]
+Arguments: 100, [lochierarchy#25 DESC NULLS LAST, CASE WHEN (lochierarchy#25 = 0) THEN i_category#13 END ASC NULLS FIRST, rank_within_parent#70 ASC NULLS FIRST], [gross_margin#22, i_category#13, i_class#12, lochierarchy#25, rank_within_parent#70]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/explain.txt
index d3e616eeda399..ecf954ef3c422 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/explain.txt
@@ -239,7 +239,7 @@ Input [6]: [gross_margin#22, i_category#10, i_class#9, lochierarchy#25, _w0#68,
(41) TakeOrderedAndProject
Input [5]: [gross_margin#22, i_category#10, i_class#9, lochierarchy#25, rank_within_parent#70]
-Arguments: 100, 0, [lochierarchy#25 DESC NULLS LAST, CASE WHEN (lochierarchy#25 = 0) THEN i_category#10 END ASC NULLS FIRST, rank_within_parent#70 ASC NULLS FIRST], [gross_margin#22, i_category#10, i_class#9, lochierarchy#25, rank_within_parent#70]
+Arguments: 100, [lochierarchy#25 DESC NULLS LAST, CASE WHEN (lochierarchy#25 = 0) THEN i_category#10 END ASC NULLS FIRST, rank_within_parent#70 ASC NULLS FIRST], [gross_margin#22, i_category#10, i_class#9, lochierarchy#25, rank_within_parent#70]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.sf100/explain.txt
index 6e9570ac21a48..4d6001619fe9b 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.sf100/explain.txt
@@ -277,7 +277,7 @@ Input [16]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_ye
(52) TakeOrderedAndProject
Input [7]: [i_category#16, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#22, psum#49, nsum#50]
-Arguments: 100, 0, [(sum_sales#22 - avg_monthly_sales#26) ASC NULLS FIRST, d_moy#8 ASC NULLS FIRST], [i_category#16, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#22, psum#49, nsum#50]
+Arguments: 100, [(sum_sales#22 - avg_monthly_sales#26) ASC NULLS FIRST, d_moy#8 ASC NULLS FIRST], [i_category#16, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#22, psum#49, nsum#50]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/explain.txt
index 431765c39d499..712a9f8c7ea3b 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/explain.txt
@@ -242,7 +242,7 @@ Input [16]: [i_category#3, i_brand#2, s_store_name#14, s_company_name#15, d_year
(45) TakeOrderedAndProject
Input [7]: [i_category#3, d_year#11, d_moy#12, avg_monthly_sales#25, sum_sales#21, psum#47, nsum#48]
-Arguments: 100, 0, [(sum_sales#21 - avg_monthly_sales#25) ASC NULLS FIRST, d_moy#12 ASC NULLS FIRST], [i_category#3, d_year#11, d_moy#12, avg_monthly_sales#25, sum_sales#21, psum#47, nsum#48]
+Arguments: 100, [(sum_sales#21 - avg_monthly_sales#25) ASC NULLS FIRST, d_moy#12 ASC NULLS FIRST], [i_category#3, d_year#11, d_moy#12, avg_monthly_sales#25, sum_sales#21, psum#47, nsum#48]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.sf100/explain.txt
index bddaba6017b02..c174a4f161788 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.sf100/explain.txt
@@ -469,7 +469,7 @@ Results [5]: [channel#39, item#33, return_ratio#34, return_rank#37, currency_ran
(86) TakeOrderedAndProject
Input [5]: [channel#39, item#33, return_ratio#34, return_rank#37, currency_rank#38]
-Arguments: 100, 0, [channel#39 ASC NULLS FIRST, return_rank#37 ASC NULLS FIRST, currency_rank#38 ASC NULLS FIRST, item#33 ASC NULLS FIRST], [channel#39, item#33, return_ratio#34, return_rank#37, currency_rank#38]
+Arguments: 100, [channel#39 ASC NULLS FIRST, return_rank#37 ASC NULLS FIRST, currency_rank#38 ASC NULLS FIRST, item#33 ASC NULLS FIRST], [channel#39, item#33, return_ratio#34, return_rank#37, currency_rank#38]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/explain.txt
index ae71bfbac6de8..ddb9caf89cbde 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/explain.txt
@@ -424,7 +424,7 @@ Results [5]: [channel#38, item#32, return_ratio#33, return_rank#36, currency_ran
(77) TakeOrderedAndProject
Input [5]: [channel#38, item#32, return_ratio#33, return_rank#36, currency_rank#37]
-Arguments: 100, 0, [channel#38 ASC NULLS FIRST, return_rank#36 ASC NULLS FIRST, currency_rank#37 ASC NULLS FIRST, item#32 ASC NULLS FIRST], [channel#38, item#32, return_ratio#33, return_rank#36, currency_rank#37]
+Arguments: 100, [channel#38 ASC NULLS FIRST, return_rank#36 ASC NULLS FIRST, currency_rank#37 ASC NULLS FIRST, item#32 ASC NULLS FIRST], [channel#38, item#32, return_ratio#33, return_rank#36, currency_rank#37]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.sf100/explain.txt
index 7c0c4f63bd4fb..740ea0f9ebbd2 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.sf100/explain.txt
@@ -386,7 +386,7 @@ Condition : ((isnotnull(web_cumulative#72) AND isnotnull(store_cumulative#73)) A
(70) TakeOrderedAndProject
Input [6]: [item_sk#56, d_date#57, web_sales#58, store_sales#59, web_cumulative#72, store_cumulative#73]
-Arguments: 100, 0, [item_sk#56 ASC NULLS FIRST, d_date#57 ASC NULLS FIRST], [item_sk#56, d_date#57, web_sales#58, store_sales#59, web_cumulative#72, store_cumulative#73]
+Arguments: 100, [item_sk#56 ASC NULLS FIRST, d_date#57 ASC NULLS FIRST], [item_sk#56, d_date#57, web_sales#58, store_sales#59, web_cumulative#72, store_cumulative#73]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/explain.txt
index 839277f4bf3d1..cf86cd670456f 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/explain.txt
@@ -371,7 +371,7 @@ Condition : ((isnotnull(web_cumulative#73) AND isnotnull(store_cumulative#74)) A
(67) TakeOrderedAndProject
Input [6]: [item_sk#56, d_date#57, web_sales#58, store_sales#59, web_cumulative#73, store_cumulative#74]
-Arguments: 100, 0, [item_sk#56 ASC NULLS FIRST, d_date#57 ASC NULLS FIRST], [item_sk#56, d_date#57, web_sales#58, store_sales#59, web_cumulative#73, store_cumulative#74]
+Arguments: 100, [item_sk#56 ASC NULLS FIRST, d_date#57 ASC NULLS FIRST], [item_sk#56, d_date#57, web_sales#58, store_sales#59, web_cumulative#73, store_cumulative#74]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.sf100/explain.txt
index 1d27ee0e8f6cd..e07d4460643c3 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.sf100/explain.txt
@@ -277,7 +277,7 @@ Input [14]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, sum_sales
(52) TakeOrderedAndProject
Input [8]: [i_category#15, i_brand#14, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#21, psum#46, nsum#47]
-Arguments: 100, 0, [(sum_sales#21 - avg_monthly_sales#25) ASC NULLS FIRST, d_year#7 ASC NULLS FIRST], [i_category#15, i_brand#14, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#21, psum#46, nsum#47]
+Arguments: 100, [(sum_sales#21 - avg_monthly_sales#25) ASC NULLS FIRST, d_year#7 ASC NULLS FIRST], [i_category#15, i_brand#14, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#21, psum#46, nsum#47]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/explain.txt
index dfcb28de5e336..1cd7e4928cc0a 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/explain.txt
@@ -242,7 +242,7 @@ Input [14]: [i_category#3, i_brand#2, cc_name#14, d_year#11, d_moy#12, sum_sales
(45) TakeOrderedAndProject
Input [8]: [i_category#3, i_brand#2, d_year#11, d_moy#12, avg_monthly_sales#24, sum_sales#20, psum#44, nsum#45]
-Arguments: 100, 0, [(sum_sales#20 - avg_monthly_sales#24) ASC NULLS FIRST, d_year#11 ASC NULLS FIRST], [i_category#3, i_brand#2, d_year#11, d_moy#12, avg_monthly_sales#24, sum_sales#20, psum#44, nsum#45]
+Arguments: 100, [(sum_sales#20 - avg_monthly_sales#24) ASC NULLS FIRST, d_year#11 ASC NULLS FIRST], [i_category#3, i_brand#2, d_year#11, d_moy#12, avg_monthly_sales#24, sum_sales#20, psum#44, nsum#45]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.sf100/explain.txt
index 64ad1725891b9..f6795596efb99 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.sf100/explain.txt
@@ -514,7 +514,7 @@ Results [5]: [channel#39, id#40, sales#151, returns#152, profit#153]
(90) TakeOrderedAndProject
Input [5]: [channel#39, id#40, sales#151, returns#152, profit#153]
-Arguments: 100, 0, [channel#39 ASC NULLS FIRST, id#40 ASC NULLS FIRST], [channel#39, id#40, sales#151, returns#152, profit#153]
+Arguments: 100, [channel#39 ASC NULLS FIRST, id#40 ASC NULLS FIRST], [channel#39, id#40, sales#151, returns#152, profit#153]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/explain.txt
index 230b85eb47e03..db39a5768b988 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/explain.txt
@@ -499,7 +499,7 @@ Results [5]: [channel#39, id#40, sales#150, returns#151, profit#152]
(87) TakeOrderedAndProject
Input [5]: [channel#39, id#40, sales#150, returns#151, profit#152]
-Arguments: 100, 0, [channel#39 ASC NULLS FIRST, id#40 ASC NULLS FIRST], [channel#39, id#40, sales#150, returns#151, profit#152]
+Arguments: 100, [channel#39 ASC NULLS FIRST, id#40 ASC NULLS FIRST], [channel#39, id#40, sales#150, returns#151, profit#152]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.sf100/explain.txt
index 648dcf6b9ac8f..25c78871aedf5 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.sf100/explain.txt
@@ -251,7 +251,7 @@ Condition : (cnt#33 >= 10)
(45) TakeOrderedAndProject
Input [3]: [state#32, cnt#33, ca_state#22]
-Arguments: 100, 0, [cnt#33 ASC NULLS FIRST, ca_state#22 ASC NULLS FIRST], [state#32, cnt#33]
+Arguments: 100, [cnt#33 ASC NULLS FIRST, ca_state#22 ASC NULLS FIRST], [state#32, cnt#33]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/explain.txt
index 9be5e8772388e..e9b40299e3676 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/explain.txt
@@ -221,7 +221,7 @@ Condition : (cnt#31 >= 10)
(39) TakeOrderedAndProject
Input [3]: [state#30, cnt#31, ca_state#2]
-Arguments: 100, 0, [cnt#31 ASC NULLS FIRST, ca_state#2 ASC NULLS FIRST], [state#30, cnt#31]
+Arguments: 100, [cnt#31 ASC NULLS FIRST, ca_state#2 ASC NULLS FIRST], [state#30, cnt#31]
===== Subqueries =====
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 2db495350c5d4..cbedda1a7bee6 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
@@ -426,7 +426,7 @@ Condition : (rk#138 <= 100)
(70) TakeOrderedAndProject
Input [10]: [i_category#18, i_class#17, i_brand#16, i_product_name#19, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#27, rk#138]
-Arguments: 100, 0, [i_category#18 ASC NULLS FIRST, i_class#17 ASC NULLS FIRST, i_brand#16 ASC NULLS FIRST, i_product_name#19 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#27 ASC NULLS FIRST, rk#138 ASC NULLS FIRST], [i_category#18, i_class#17, i_brand#16, i_product_name#19, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#27, rk#138]
+Arguments: 100, [i_category#18 ASC NULLS FIRST, i_class#17 ASC NULLS FIRST, i_brand#16 ASC NULLS FIRST, i_product_name#19 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#27 ASC NULLS FIRST, rk#138 ASC NULLS FIRST], [i_category#18, i_class#17, i_brand#16, i_product_name#19, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#27, rk#138]
===== Subqueries =====
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 65e590216b11f..fd3033f54554e 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
@@ -411,7 +411,7 @@ Condition : (rk#137 <= 100)
(67) TakeOrderedAndProject
Input [10]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#26, rk#137]
-Arguments: 100, 0, [i_category#17 ASC NULLS FIRST, i_class#16 ASC NULLS FIRST, i_brand#15 ASC NULLS FIRST, i_product_name#18 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#26 ASC NULLS FIRST, rk#137 ASC NULLS FIRST], [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#26, rk#137]
+Arguments: 100, [i_category#17 ASC NULLS FIRST, i_class#16 ASC NULLS FIRST, i_brand#15 ASC NULLS FIRST, i_product_name#18 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#26 ASC NULLS FIRST, rk#137 ASC NULLS FIRST], [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#26, rk#137]
===== Subqueries =====
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 0e8c3ac9f8ae1..015dc51d2aec6 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
@@ -324,7 +324,7 @@ Input [6]: [total_sum#30, s_state#8, s_county#7, lochierarchy#33, _w0#61, rank_w
(56) TakeOrderedAndProject
Input [5]: [total_sum#30, s_state#8, s_county#7, lochierarchy#33, rank_within_parent#63]
-Arguments: 100, 0, [lochierarchy#33 DESC NULLS LAST, CASE WHEN (lochierarchy#33 = 0) THEN s_state#8 END ASC NULLS FIRST, rank_within_parent#63 ASC NULLS FIRST], [total_sum#30, s_state#8, s_county#7, lochierarchy#33, rank_within_parent#63]
+Arguments: 100, [lochierarchy#33 DESC NULLS LAST, CASE WHEN (lochierarchy#33 = 0) THEN s_state#8 END ASC NULLS FIRST, rank_within_parent#63 ASC NULLS FIRST], [total_sum#30, s_state#8, s_county#7, lochierarchy#33, rank_within_parent#63]
===== Subqueries =====
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 5db98387feb61..a67d7e109e7fe 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
@@ -324,7 +324,7 @@ Input [6]: [total_sum#30, s_state#8, s_county#7, lochierarchy#33, _w0#61, rank_w
(56) TakeOrderedAndProject
Input [5]: [total_sum#30, s_state#8, s_county#7, lochierarchy#33, rank_within_parent#63]
-Arguments: 100, 0, [lochierarchy#33 DESC NULLS LAST, CASE WHEN (lochierarchy#33 = 0) THEN s_state#8 END ASC NULLS FIRST, rank_within_parent#63 ASC NULLS FIRST], [total_sum#30, s_state#8, s_county#7, lochierarchy#33, rank_within_parent#63]
+Arguments: 100, [lochierarchy#33 DESC NULLS LAST, CASE WHEN (lochierarchy#33 = 0) THEN s_state#8 END ASC NULLS FIRST, rank_within_parent#63 ASC NULLS FIRST], [total_sum#30, s_state#8, s_county#7, lochierarchy#33, rank_within_parent#63]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/explain.txt
index 4169bbb51d9d6..e5e42f2be1366 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/explain.txt
@@ -383,7 +383,7 @@ Results [6]: [i_item_desc#21, w_warehouse_name#33, d_week_seq#25, count(1)#46 AS
(70) TakeOrderedAndProject
Input [6]: [i_item_desc#21, w_warehouse_name#33, d_week_seq#25, no_promo#47, promo#48, total_cnt#49]
-Arguments: 100, 0, [total_cnt#49 DESC NULLS LAST, i_item_desc#21 ASC NULLS FIRST, w_warehouse_name#33 ASC NULLS FIRST, d_week_seq#25 ASC NULLS FIRST], [i_item_desc#21, w_warehouse_name#33, d_week_seq#25, no_promo#47, promo#48, total_cnt#49]
+Arguments: 100, [total_cnt#49 DESC NULLS LAST, i_item_desc#21 ASC NULLS FIRST, w_warehouse_name#33 ASC NULLS FIRST, d_week_seq#25 ASC NULLS FIRST], [i_item_desc#21, w_warehouse_name#33, d_week_seq#25, no_promo#47, promo#48, total_cnt#49]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/explain.txt
index ae21c596d5bd0..33ef0ab30ebc9 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/explain.txt
@@ -386,7 +386,7 @@ Results [6]: [i_item_desc#19, w_warehouse_name#16, d_week_seq#29, count(1)#46 AS
(70) TakeOrderedAndProject
Input [6]: [i_item_desc#19, w_warehouse_name#16, d_week_seq#29, no_promo#47, promo#48, total_cnt#49]
-Arguments: 100, 0, [total_cnt#49 DESC NULLS LAST, i_item_desc#19 ASC NULLS FIRST, w_warehouse_name#16 ASC NULLS FIRST, d_week_seq#29 ASC NULLS FIRST], [i_item_desc#19, w_warehouse_name#16, d_week_seq#29, no_promo#47, promo#48, total_cnt#49]
+Arguments: 100, [total_cnt#49 DESC NULLS LAST, i_item_desc#19 ASC NULLS FIRST, w_warehouse_name#16 ASC NULLS FIRST, d_week_seq#29 ASC NULLS FIRST], [i_item_desc#19, w_warehouse_name#16, d_week_seq#29, no_promo#47, promo#48, total_cnt#49]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.sf100/explain.txt
index 1ad1831fcd87d..3b21d6fd03243 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.sf100/explain.txt
@@ -441,7 +441,7 @@ Input [9]: [customer_id#17, year_total#18, customer_id#34, customer_first_name#3
(80) TakeOrderedAndProject
Input [3]: [customer_id#34, customer_first_name#35, customer_last_name#36]
-Arguments: 100, 0, [customer_first_name#35 ASC NULLS FIRST, customer_id#34 ASC NULLS FIRST, customer_last_name#36 ASC NULLS FIRST], [customer_id#34, customer_first_name#35, customer_last_name#36]
+Arguments: 100, [customer_first_name#35 ASC NULLS FIRST, customer_id#34 ASC NULLS FIRST, customer_last_name#36 ASC NULLS FIRST], [customer_id#34, customer_first_name#35, customer_last_name#36]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/explain.txt
index 5ff8bb0039112..a3a369626beb5 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/explain.txt
@@ -410,7 +410,7 @@ Input [9]: [customer_id#16, year_total#17, customer_id#32, customer_first_name#3
(72) TakeOrderedAndProject
Input [3]: [customer_id#32, customer_first_name#33, customer_last_name#34]
-Arguments: 100, 0, [customer_first_name#33 ASC NULLS FIRST, customer_id#32 ASC NULLS FIRST, customer_last_name#34 ASC NULLS FIRST], [customer_id#32, customer_first_name#33, customer_last_name#34]
+Arguments: 100, [customer_first_name#33 ASC NULLS FIRST, customer_id#32 ASC NULLS FIRST, customer_last_name#34 ASC NULLS FIRST], [customer_id#32, customer_first_name#33, customer_last_name#34]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.sf100/explain.txt
index e32c4f29ee131..7e213628bfc18 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.sf100/explain.txt
@@ -695,7 +695,7 @@ Input [14]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact
(129) TakeOrderedAndProject
Input [10]: [prev_year#137, year#138, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#139, curr_yr_cnt#140, sales_cnt_diff#141, sales_amt_diff#142]
-Arguments: 100, 0, [sales_cnt_diff#141 ASC NULLS FIRST, sales_amt_diff#142 ASC NULLS FIRST], [prev_year#137, year#138, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#139, curr_yr_cnt#140, sales_cnt_diff#141, sales_amt_diff#142]
+Arguments: 100, [sales_cnt_diff#141 ASC NULLS FIRST, sales_amt_diff#142 ASC NULLS FIRST], [prev_year#137, year#138, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#139, curr_yr_cnt#140, sales_cnt_diff#141, sales_amt_diff#142]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/explain.txt
index e32c4f29ee131..7e213628bfc18 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/explain.txt
@@ -695,7 +695,7 @@ Input [14]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact
(129) TakeOrderedAndProject
Input [10]: [prev_year#137, year#138, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#139, curr_yr_cnt#140, sales_cnt_diff#141, sales_amt_diff#142]
-Arguments: 100, 0, [sales_cnt_diff#141 ASC NULLS FIRST, sales_amt_diff#142 ASC NULLS FIRST], [prev_year#137, year#138, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#139, curr_yr_cnt#140, sales_cnt_diff#141, sales_amt_diff#142]
+Arguments: 100, [sales_cnt_diff#141 ASC NULLS FIRST, sales_amt_diff#142 ASC NULLS FIRST], [prev_year#137, year#138, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#139, curr_yr_cnt#140, sales_cnt_diff#141, sales_amt_diff#142]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.sf100/explain.txt
index 461007be5b3a9..27945848a133a 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.sf100/explain.txt
@@ -572,7 +572,7 @@ Results [5]: [channel#34, id#35, sales#121, returns#122, profit#123]
(98) TakeOrderedAndProject
Input [5]: [channel#34, id#35, sales#121, returns#122, profit#123]
-Arguments: 100, 0, [channel#34 ASC NULLS FIRST, id#35 ASC NULLS FIRST], [channel#34, id#35, sales#121, returns#122, profit#123]
+Arguments: 100, [channel#34 ASC NULLS FIRST, id#35 ASC NULLS FIRST], [channel#34, id#35, sales#121, returns#122, profit#123]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/explain.txt
index 65661a80f9f84..74f61a018b497 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/explain.txt
@@ -572,7 +572,7 @@ Results [5]: [channel#34, id#35, sales#121, returns#122, profit#123]
(98) TakeOrderedAndProject
Input [5]: [channel#34, id#35, sales#121, returns#122, profit#123]
-Arguments: 100, 0, [channel#34 ASC NULLS FIRST, id#35 ASC NULLS FIRST], [channel#34, id#35, sales#121, returns#122, profit#123]
+Arguments: 100, [channel#34 ASC NULLS FIRST, id#35 ASC NULLS FIRST], [channel#34, id#35, sales#121, returns#122, profit#123]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.sf100/explain.txt
index bf191ba233304..c54f269af69b6 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.sf100/explain.txt
@@ -387,7 +387,7 @@ Input [15]: [ss_sold_year#26, ss_item_sk#1, ss_customer_sk#2, ss_qty#27, ss_wc#2
(70) TakeOrderedAndProject
Input [13]: [ratio#88, store_qty#89, store_wholesale_cost#90, store_sales_price#91, other_chan_qty#92, other_chan_wholesale_cost#93, other_chan_sales_price#94, ss_sold_year#26, ss_item_sk#1, ss_customer_sk#2, ss_qty#27, ss_wc#28, ss_sp#29]
-Arguments: 100, 0, [ss_sold_year#26 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST, ss_qty#27 DESC NULLS LAST, ss_wc#28 DESC NULLS LAST, ss_sp#29 DESC NULLS LAST, other_chan_qty#92 ASC NULLS FIRST, other_chan_wholesale_cost#93 ASC NULLS FIRST, other_chan_sales_price#94 ASC NULLS FIRST, ratio#88 ASC NULLS FIRST], [ratio#88, store_qty#89, store_wholesale_cost#90, store_sales_price#91, other_chan_qty#92, other_chan_wholesale_cost#93, other_chan_sales_price#94]
+Arguments: 100, [ss_sold_year#26 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST, ss_qty#27 DESC NULLS LAST, ss_wc#28 DESC NULLS LAST, ss_sp#29 DESC NULLS LAST, other_chan_qty#92 ASC NULLS FIRST, other_chan_wholesale_cost#93 ASC NULLS FIRST, other_chan_sales_price#94 ASC NULLS FIRST, ratio#88 ASC NULLS FIRST], [ratio#88, store_qty#89, store_wholesale_cost#90, store_sales_price#91, other_chan_qty#92, other_chan_wholesale_cost#93, other_chan_sales_price#94]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/explain.txt
index bf191ba233304..c54f269af69b6 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/explain.txt
@@ -387,7 +387,7 @@ Input [15]: [ss_sold_year#26, ss_item_sk#1, ss_customer_sk#2, ss_qty#27, ss_wc#2
(70) TakeOrderedAndProject
Input [13]: [ratio#88, store_qty#89, store_wholesale_cost#90, store_sales_price#91, other_chan_qty#92, other_chan_wholesale_cost#93, other_chan_sales_price#94, ss_sold_year#26, ss_item_sk#1, ss_customer_sk#2, ss_qty#27, ss_wc#28, ss_sp#29]
-Arguments: 100, 0, [ss_sold_year#26 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST, ss_qty#27 DESC NULLS LAST, ss_wc#28 DESC NULLS LAST, ss_sp#29 DESC NULLS LAST, other_chan_qty#92 ASC NULLS FIRST, other_chan_wholesale_cost#93 ASC NULLS FIRST, other_chan_sales_price#94 ASC NULLS FIRST, ratio#88 ASC NULLS FIRST], [ratio#88, store_qty#89, store_wholesale_cost#90, store_sales_price#91, other_chan_qty#92, other_chan_wholesale_cost#93, other_chan_sales_price#94]
+Arguments: 100, [ss_sold_year#26 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST, ss_qty#27 DESC NULLS LAST, ss_wc#28 DESC NULLS LAST, ss_sp#29 DESC NULLS LAST, other_chan_qty#92 ASC NULLS FIRST, other_chan_wholesale_cost#93 ASC NULLS FIRST, other_chan_sales_price#94 ASC NULLS FIRST, ratio#88 ASC NULLS FIRST], [ratio#88, store_qty#89, store_wholesale_cost#90, store_sales_price#91, other_chan_qty#92, other_chan_wholesale_cost#93, other_chan_sales_price#94]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.sf100/explain.txt
index 30fdcd756313b..ec5fc20a86a73 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.sf100/explain.txt
@@ -674,7 +674,7 @@ Results [5]: [channel#40, id#41, sales#139, returns#140, profit#141]
(120) TakeOrderedAndProject
Input [5]: [channel#40, id#41, sales#139, returns#140, profit#141]
-Arguments: 100, 0, [channel#40 ASC NULLS FIRST, id#41 ASC NULLS FIRST], [channel#40, id#41, sales#139, returns#140, profit#141]
+Arguments: 100, [channel#40 ASC NULLS FIRST, id#41 ASC NULLS FIRST], [channel#40, id#41, sales#139, returns#140, profit#141]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/explain.txt
index 43a97b4cb7ff5..3ce8c5db2b207 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/explain.txt
@@ -674,7 +674,7 @@ Results [5]: [channel#40, id#41, sales#139, returns#140, profit#141]
(120) TakeOrderedAndProject
Input [5]: [channel#40, id#41, sales#139, returns#140, profit#141]
-Arguments: 100, 0, [channel#40 ASC NULLS FIRST, id#41 ASC NULLS FIRST], [channel#40, id#41, sales#139, returns#140, profit#141]
+Arguments: 100, [channel#40 ASC NULLS FIRST, id#41 ASC NULLS FIRST], [channel#40, id#41, sales#139, returns#140, profit#141]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.sf100/explain.txt
index 19be39c47ba65..1dd3dc76a8f56 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.sf100/explain.txt
@@ -201,7 +201,7 @@ Input [6]: [total_sum#14, i_category#8, i_class#7, lochierarchy#17, _w0#45, rank
(34) TakeOrderedAndProject
Input [5]: [total_sum#14, i_category#8, i_class#7, lochierarchy#17, rank_within_parent#47]
-Arguments: 100, 0, [lochierarchy#17 DESC NULLS LAST, CASE WHEN (lochierarchy#17 = 0) THEN i_category#8 END ASC NULLS FIRST, rank_within_parent#47 ASC NULLS FIRST], [total_sum#14, i_category#8, i_class#7, lochierarchy#17, rank_within_parent#47]
+Arguments: 100, [lochierarchy#17 DESC NULLS LAST, CASE WHEN (lochierarchy#17 = 0) THEN i_category#8 END ASC NULLS FIRST, rank_within_parent#47 ASC NULLS FIRST], [total_sum#14, i_category#8, i_class#7, lochierarchy#17, rank_within_parent#47]
===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/explain.txt
index 19be39c47ba65..1dd3dc76a8f56 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/explain.txt
@@ -201,7 +201,7 @@ Input [6]: [total_sum#14, i_category#8, i_class#7, lochierarchy#17, _w0#45, rank
(34) TakeOrderedAndProject
Input [5]: [total_sum#14, i_category#8, i_class#7, lochierarchy#17, rank_within_parent#47]
-Arguments: 100, 0, [lochierarchy#17 DESC NULLS LAST, CASE WHEN (lochierarchy#17 = 0) THEN i_category#8 END ASC NULLS FIRST, rank_within_parent#47 ASC NULLS FIRST], [total_sum#14, i_category#8, i_class#7, lochierarchy#17, rank_within_parent#47]
+Arguments: 100, [lochierarchy#17 DESC NULLS LAST, CASE WHEN (lochierarchy#17 = 0) THEN i_category#8 END ASC NULLS FIRST, rank_within_parent#47 ASC NULLS FIRST], [total_sum#14, i_category#8, i_class#7, lochierarchy#17, rank_within_parent#47]
===== Subqueries =====
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala
index 2fd59932d230e..2e2fdd17207bb 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala
@@ -603,6 +603,30 @@ class DataFrameSuite extends QueryTest
)
}
+ test("offset") {
+ checkAnswer(
+ testData.offset(90),
+ testData.collect().drop(90).toSeq)
+
+ checkAnswer(
+ arrayData.toDF().offset(99),
+ arrayData.collect().drop(99).map(r => Row.fromSeq(r.productIterator.toSeq)))
+
+ checkAnswer(
+ mapData.toDF().offset(99),
+ mapData.collect().drop(99).map(r => Row.fromSeq(r.productIterator.toSeq)))
+ }
+
+ test("limit with offset") {
+ checkAnswer(
+ testData.limit(10).offset(5),
+ testData.take(10).drop(5).toSeq)
+
+ checkAnswer(
+ testData.offset(5).limit(10),
+ testData.take(15).drop(5).toSeq)
+ }
+
test("udf") {
val foo = udf((a: Int, b: String) => a.toString + b)
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/TakeOrderedAndProjectSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/TakeOrderedAndProjectSuite.scala
index 0f6c808b16708..6ec5c6287eed1 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/TakeOrderedAndProjectSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/TakeOrderedAndProjectSuite.scala
@@ -66,7 +66,7 @@ class TakeOrderedAndProjectSuite extends SparkPlanTest with SharedSparkSession {
checkThatPlansAgree(
generateRandomInputData(n, m),
input =>
- noOpFilter(TakeOrderedAndProjectExec(limit, 0, sortOrder, input.output, input)),
+ noOpFilter(TakeOrderedAndProjectExec(limit, sortOrder, input.output, input)),
input =>
GlobalLimitExec(limit,
LocalLimitExec(limit,
@@ -83,7 +83,7 @@ class TakeOrderedAndProjectSuite extends SparkPlanTest with SharedSparkSession {
generateRandomInputData(n, m),
input =>
noOpFilter(
- TakeOrderedAndProjectExec(limit, 0, sortOrder, Seq(input.output.last), input)),
+ TakeOrderedAndProjectExec(limit, sortOrder, Seq(input.output.last), input)),
input =>
GlobalLimitExec(limit,
LocalLimitExec(limit,
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2StrategySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2StrategySuite.scala
index 6296da47cca51..1a5a382afdc6b 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2StrategySuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2StrategySuite.scala
@@ -37,7 +37,7 @@ class DataSourceV2StrategySuite extends PlanTest with SharedSparkSession {
*/
def testTranslateFilter(catalystFilter: Expression, result: Option[Predicate]): Unit = {
assertResult(result) {
- DataSourceV2Strategy.translateFilterV2(catalystFilter, true)
+ DataSourceV2Strategy.translateFilterV2(catalystFilter)
}
}
}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCV2Suite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCV2Suite.scala
index eb8ca9ea042a0..2a5f88234513a 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCV2Suite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCV2Suite.scala
@@ -23,11 +23,11 @@ import java.util.Properties
import org.apache.spark.{SparkConf, SparkException}
import org.apache.spark.sql.{AnalysisException, DataFrame, ExplainSuiteHelper, QueryTest, Row}
import org.apache.spark.sql.catalyst.analysis.CannotReplaceMissingTableException
-import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, GlobalLimit, LocalLimit, Sort}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, GlobalLimit, LocalLimit, Offset, Sort}
import org.apache.spark.sql.connector.IntegralAverage
import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2ScanRelation, V1ScanWrapper}
import org.apache.spark.sql.execution.datasources.v2.jdbc.JDBCTableCatalog
-import org.apache.spark.sql.functions.{abs, avg, coalesce, count, count_distinct, exp, lit, log => ln, not, pow, sqrt, sum, udf, when}
+import org.apache.spark.sql.functions.{abs, acos, asin, atan, atan2, avg, ceil, coalesce, cos, cosh, count, count_distinct, degrees, exp, floor, lit, log => logarithm, log10, not, pow, radians, round, signum, sin, sinh, sqrt, sum, tan, tanh, udf, when}
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.util.Utils
@@ -45,6 +45,7 @@ class JDBCV2Suite extends QueryTest with SharedSparkSession with ExplainSuiteHel
.set("spark.sql.catalog.h2.driver", "org.h2.Driver")
.set("spark.sql.catalog.h2.pushDownAggregate", "true")
.set("spark.sql.catalog.h2.pushDownLimit", "true")
+ .set("spark.sql.catalog.h2.pushDownOffset", "true")
private def withConnection[T](f: Connection => T): T = {
val conn = DriverManager.getConnection(url, new Properties())
@@ -82,9 +83,10 @@ class JDBCV2Suite extends QueryTest with SharedSparkSession with ExplainSuiteHel
conn.prepareStatement(
"INSERT INTO \"test\".\"employee\" VALUES (6, 'jen', 12000, 1200, true)").executeUpdate()
conn.prepareStatement(
- "CREATE TABLE \"test\".\"dept\" (\"dept_id\" INTEGER NOT NULL)").executeUpdate()
- conn.prepareStatement("INSERT INTO \"test\".\"dept\" VALUES (1)").executeUpdate()
- conn.prepareStatement("INSERT INTO \"test\".\"dept\" VALUES (2)").executeUpdate()
+ "CREATE TABLE \"test\".\"dept\" (\"dept_id\" INTEGER NOT NULL, \"dept_id1\" INTEGER)")
+ .executeUpdate()
+ conn.prepareStatement("INSERT INTO \"test\".\"dept\" VALUES (1, 1)").executeUpdate()
+ conn.prepareStatement("INSERT INTO \"test\".\"dept\" VALUES (2, 1)").executeUpdate()
// scalastyle:off
conn.prepareStatement(
@@ -120,10 +122,10 @@ class JDBCV2Suite extends QueryTest with SharedSparkSession with ExplainSuiteHel
checkAnswer(sql("SELECT name, id FROM h2.test.people"), Seq(Row("fred", 1), Row("mary", 2)))
}
- private def checkPushedInfo(df: DataFrame, expectedPlanFragment: String): Unit = {
+ private def checkPushedInfo(df: DataFrame, expectedPlanFragment: String*): Unit = {
df.queryExecution.optimizedPlan.collect {
case _: DataSourceV2ScanRelation =>
- checkKeywordsExistsInExplain(df, expectedPlanFragment)
+ checkKeywordsExistsInExplain(df, expectedPlanFragment: _*)
}
}
@@ -206,6 +208,355 @@ class JDBCV2Suite extends QueryTest with SharedSparkSession with ExplainSuiteHel
checkAnswer(df5, Seq(Row(10000.00, 1000.0, "amy")))
}
+ private def checkOffsetRemoved(df: DataFrame, removed: Boolean = true): Unit = {
+ val offsets = df.queryExecution.optimizedPlan.collect {
+ case offset: Offset => offset
+ }
+ if (removed) {
+ assert(offsets.isEmpty)
+ } else {
+ assert(offsets.nonEmpty)
+ }
+ }
+
+ test("simple scan with OFFSET") {
+ val df1 = spark.read
+ .table("h2.test.employee")
+ .where($"dept" === 1)
+ .offset(1)
+ checkOffsetRemoved(df1)
+ checkPushedInfo(df1,
+ "PushedFilters: [DEPT IS NOT NULL, DEPT = 1], PushedOffset: OFFSET 1,")
+ checkAnswer(df1, Seq(Row(1, "cathy", 9000.00, 1200.0, false)))
+
+ val df2 = spark.read
+ .option("pushDownOffset", "false")
+ .table("h2.test.employee")
+ .where($"dept" === 1)
+ .offset(1)
+ checkOffsetRemoved(df2, false)
+ checkPushedInfo(df2,
+ "PushedFilters: [DEPT IS NOT NULL, DEPT = 1], ReadSchema:")
+ checkAnswer(df2, Seq(Row(1, "cathy", 9000.00, 1200.0, false)))
+
+ val df3 = spark.read
+ .table("h2.test.employee")
+ .where($"dept" === 1)
+ .sort($"salary")
+ .offset(1)
+ checkOffsetRemoved(df3, false)
+ checkPushedInfo(df3,
+ "PushedFilters: [DEPT IS NOT NULL, DEPT = 1], ReadSchema:")
+ checkAnswer(df3, Seq(Row(1, "amy", 10000.00, 1000.0, true)))
+
+ val df4 = spark.read
+ .option("partitionColumn", "dept")
+ .option("lowerBound", "0")
+ .option("upperBound", "2")
+ .option("numPartitions", "2")
+ .table("h2.test.employee")
+ .filter($"dept" > 1)
+ .offset(1)
+ checkOffsetRemoved(df4, false)
+ checkPushedInfo(df4, "PushedFilters: [DEPT IS NOT NULL, DEPT > 1], ReadSchema:")
+ checkAnswer(df4, Seq(Row(2, "david", 10000, 1300, true), Row(6, "jen", 12000, 1200, true)))
+
+ val df5 = spark.read
+ .table("h2.test.employee")
+ .groupBy("DEPT").sum("SALARY")
+ .offset(1)
+ checkOffsetRemoved(df5, false)
+ checkPushedInfo(df5,
+ "PushedAggregates: [SUM(SALARY)], PushedFilters: [], PushedGroupByExpressions: [DEPT], ")
+ checkAnswer(df5, Seq(Row(2, 22000.00), Row(6, 12000.00)))
+
+ val name = udf { (x: String) => x.matches("cat|dav|amy") }
+ val sub = udf { (x: String) => x.substring(0, 3) }
+ val df6 = spark.read
+ .table("h2.test.employee")
+ .select($"SALARY", $"BONUS", sub($"NAME").as("shortName"))
+ .filter(name($"shortName"))
+ .offset(1)
+ checkOffsetRemoved(df6, false)
+ // OFFSET is pushed down only if all the filters are pushed down
+ checkPushedInfo(df6, "PushedFilters: [], ")
+ checkAnswer(df6, Seq(Row(10000.00, 1300.0, "dav"), Row(9000.00, 1200.0, "cat")))
+ }
+
+ test("simple scan with LIMIT and OFFSET") {
+ val df1 = spark.read
+ .table("h2.test.employee")
+ .where($"dept" === 1)
+ .limit(2)
+ .offset(1)
+ checkLimitRemoved(df1)
+ checkOffsetRemoved(df1)
+ checkPushedInfo(df1,
+ "PushedFilters: [DEPT IS NOT NULL, DEPT = 1], PushedLimit: LIMIT 2, PushedOffset: OFFSET 1,")
+ checkAnswer(df1, Seq(Row(1, "cathy", 9000.00, 1200.0, false)))
+
+ val df2 = spark.read
+ .option("pushDownLimit", "false")
+ .table("h2.test.employee")
+ .where($"dept" === 1)
+ .limit(2)
+ .offset(1)
+ checkLimitRemoved(df2, false)
+ checkOffsetRemoved(df2, false)
+ checkPushedInfo(df2,
+ "PushedFilters: [DEPT IS NOT NULL, DEPT = 1], ReadSchema:")
+ checkAnswer(df2, Seq(Row(1, "cathy", 9000.00, 1200.0, false)))
+
+ val df3 = spark.read
+ .option("pushDownOffset", "false")
+ .table("h2.test.employee")
+ .where($"dept" === 1)
+ .limit(2)
+ .offset(1)
+ checkLimitRemoved(df3)
+ checkOffsetRemoved(df3, false)
+ checkPushedInfo(df3,
+ "PushedFilters: [DEPT IS NOT NULL, DEPT = 1], PushedLimit: LIMIT 2, ReadSchema:")
+ checkAnswer(df3, Seq(Row(1, "cathy", 9000.00, 1200.0, false)))
+
+ val df4 = spark.read
+ .option("pushDownLimit", "false")
+ .option("pushDownOffset", "false")
+ .table("h2.test.employee")
+ .where($"dept" === 1)
+ .limit(2)
+ .offset(1)
+ checkLimitRemoved(df4, false)
+ checkOffsetRemoved(df4, false)
+ checkPushedInfo(df4,
+ "PushedFilters: [DEPT IS NOT NULL, DEPT = 1], ReadSchema:")
+ checkAnswer(df4, Seq(Row(1, "cathy", 9000.00, 1200.0, false)))
+
+ val df5 = spark.read
+ .table("h2.test.employee")
+ .where($"dept" === 1)
+ .sort($"salary")
+ .limit(2)
+ .offset(1)
+ checkLimitRemoved(df5)
+ checkOffsetRemoved(df5)
+ checkPushedInfo(df5, "PushedFilters: [DEPT IS NOT NULL, DEPT = 1], " +
+ "PushedOffset: OFFSET 1, PushedTopN: ORDER BY [SALARY ASC NULLS FIRST] LIMIT 2, ReadSchema:")
+ checkAnswer(df5, Seq(Row(1, "amy", 10000.00, 1000.0, true)))
+
+ val df6 = spark.read
+ .option("pushDownLimit", "false")
+ .table("h2.test.employee")
+ .where($"dept" === 1)
+ .sort($"salary")
+ .limit(2)
+ .offset(1)
+ checkLimitRemoved(df6, false)
+ checkOffsetRemoved(df6, false)
+ checkPushedInfo(df6, "PushedFilters: [DEPT IS NOT NULL, DEPT = 1], ReadSchema:")
+ checkAnswer(df6, Seq(Row(1, "amy", 10000.00, 1000.0, true)))
+
+ val df7 = spark.read
+ .option("pushDownOffset", "false")
+ .table("h2.test.employee")
+ .where($"dept" === 1)
+ .sort($"salary")
+ .limit(2)
+ .offset(1)
+ checkLimitRemoved(df7)
+ checkOffsetRemoved(df7, false)
+ checkPushedInfo(df7, "PushedFilters: [DEPT IS NOT NULL, DEPT = 1]," +
+ " PushedTopN: ORDER BY [SALARY ASC NULLS FIRST] LIMIT 2, ReadSchema:")
+ checkAnswer(df7, Seq(Row(1, "amy", 10000.00, 1000.0, true)))
+
+ val df8 = spark.read
+ .option("pushDownLimit", "false")
+ .option("pushDownOffset", "false")
+ .table("h2.test.employee")
+ .where($"dept" === 1)
+ .sort($"salary")
+ .limit(2)
+ .offset(1)
+ checkLimitRemoved(df8, false)
+ checkOffsetRemoved(df8, false)
+ checkPushedInfo(df8, "PushedFilters: [DEPT IS NOT NULL, DEPT = 1], ReadSchema:")
+ checkAnswer(df8, Seq(Row(1, "amy", 10000.00, 1000.0, true)))
+
+ val df9 = spark.read
+ .option("partitionColumn", "dept")
+ .option("lowerBound", "0")
+ .option("upperBound", "2")
+ .option("numPartitions", "2")
+ .table("h2.test.employee")
+ .filter($"dept" > 1)
+ .limit(2)
+ .offset(1)
+ checkLimitRemoved(df9, false)
+ checkOffsetRemoved(df9, false)
+ checkPushedInfo(df9,
+ "PushedFilters: [DEPT IS NOT NULL, DEPT > 1], PushedLimit: LIMIT 2, ReadSchema:")
+ checkAnswer(df9, Seq(Row(2, "david", 10000.00, 1300.0, true)))
+
+ val df10 = spark.read
+ .table("h2.test.employee")
+ .groupBy("DEPT").sum("SALARY")
+ .limit(2)
+ .offset(1)
+ checkLimitRemoved(df10, false)
+ checkOffsetRemoved(df10, false)
+ checkPushedInfo(df10,
+ "PushedAggregates: [SUM(SALARY)], PushedFilters: [], PushedGroupByExpressions: [DEPT], ")
+ checkAnswer(df10, Seq(Row(2, 22000.00)))
+
+ val name = udf { (x: String) => x.matches("cat|dav|amy") }
+ val sub = udf { (x: String) => x.substring(0, 3) }
+ val df11 = spark.read
+ .table("h2.test.employee")
+ .select($"SALARY", $"BONUS", sub($"NAME").as("shortName"))
+ .filter(name($"shortName"))
+ .limit(2)
+ .offset(1)
+ checkLimitRemoved(df11, false)
+ checkOffsetRemoved(df11, false)
+ checkPushedInfo(df11, "PushedFilters: [], ")
+ checkAnswer(df11, Seq(Row(9000.00, 1200.0, "cat")))
+ }
+
+ test("simple scan with OFFSET and LIMIT") {
+ val df1 = spark.read
+ .table("h2.test.employee")
+ .where($"dept" === 1)
+ .offset(1)
+ .limit(1)
+ checkLimitRemoved(df1)
+ checkOffsetRemoved(df1)
+ checkPushedInfo(df1,
+ "[DEPT IS NOT NULL, DEPT = 1], PushedLimit: LIMIT 2, PushedOffset: OFFSET 1,")
+ checkAnswer(df1, Seq(Row(1, "cathy", 9000.00, 1200.0, false)))
+
+ val df2 = spark.read
+ .option("pushDownOffset", "false")
+ .table("h2.test.employee")
+ .where($"dept" === 1)
+ .offset(1)
+ .limit(1)
+ checkLimitRemoved(df2)
+ checkOffsetRemoved(df2, false)
+ checkPushedInfo(df2,
+ "[DEPT IS NOT NULL, DEPT = 1], PushedLimit: LIMIT 2, ReadSchema:")
+ checkAnswer(df2, Seq(Row(1, "cathy", 9000.00, 1200.0, false)))
+
+ val df3 = spark.read
+ .option("pushDownLimit", "false")
+ .table("h2.test.employee")
+ .where($"dept" === 1)
+ .offset(1)
+ .limit(1)
+ checkLimitRemoved(df3, false)
+ checkOffsetRemoved(df3)
+ checkPushedInfo(df3,
+ "[DEPT IS NOT NULL, DEPT = 1], PushedOffset: OFFSET 1, ReadSchema:")
+ checkAnswer(df3, Seq(Row(1, "cathy", 9000.00, 1200.0, false)))
+
+ val df4 = spark.read
+ .option("pushDownOffset", "false")
+ .option("pushDownLimit", "false")
+ .table("h2.test.employee")
+ .where($"dept" === 1)
+ .offset(1)
+ .limit(1)
+ checkLimitRemoved(df4, false)
+ checkOffsetRemoved(df4, false)
+ checkPushedInfo(df4,
+ "[DEPT IS NOT NULL, DEPT = 1], ReadSchema:")
+ checkAnswer(df4, Seq(Row(1, "cathy", 9000.00, 1200.0, false)))
+
+ val df5 = spark.read
+ .table("h2.test.employee")
+ .where($"dept" === 1)
+ .sort($"salary")
+ .offset(1)
+ .limit(1)
+ checkLimitRemoved(df5)
+ checkOffsetRemoved(df5)
+ checkPushedInfo(df5, "PushedFilters: [DEPT IS NOT NULL, DEPT = 1], " +
+ "PushedOffset: OFFSET 1, PushedTopN: ORDER BY [SALARY ASC NULLS FIRST] LIMIT 2, ReadSchema:")
+ checkAnswer(df5, Seq(Row(1, "amy", 10000.00, 1000.0, true)))
+
+ val df6 = spark.read
+ .option("pushDownOffset", "false")
+ .table("h2.test.employee")
+ .where($"dept" === 1)
+ .sort($"salary")
+ .offset(1)
+ .limit(1)
+ checkLimitRemoved(df6)
+ checkOffsetRemoved(df6, false)
+ checkPushedInfo(df6, "[DEPT IS NOT NULL, DEPT = 1]," +
+ " PushedTopN: ORDER BY [SALARY ASC NULLS FIRST] LIMIT 2, ReadSchema:")
+ checkAnswer(df6, Seq(Row(1, "amy", 10000.00, 1000.0, true)))
+
+ val df7 = spark.read
+ .option("pushDownLimit", "false")
+ .table("h2.test.employee")
+ .where($"dept" === 1)
+ .sort($"salary")
+ .offset(1)
+ .limit(1)
+ checkLimitRemoved(df7, false)
+ checkOffsetRemoved(df7, false)
+ checkPushedInfo(df7, "PushedFilters: [DEPT IS NOT NULL, DEPT = 1], ReadSchema:")
+ checkAnswer(df7, Seq(Row(1, "amy", 10000.00, 1000.0, true)))
+
+ val df8 = spark.read
+ .option("pushDownOffset", "false")
+ .option("pushDownLimit", "false")
+ .table("h2.test.employee")
+ .where($"dept" === 1)
+ .sort($"salary")
+ .offset(1)
+ .limit(1)
+ checkLimitRemoved(df8, false)
+ checkOffsetRemoved(df8, false)
+ checkPushedInfo(df8, "PushedFilters: [DEPT IS NOT NULL, DEPT = 1], ReadSchema:")
+ checkAnswer(df8, Seq(Row(1, "amy", 10000.00, 1000.0, true)))
+
+ val df9 = spark.read
+ .option("partitionColumn", "dept")
+ .option("lowerBound", "0")
+ .option("upperBound", "2")
+ .option("numPartitions", "2")
+ .table("h2.test.employee")
+ .filter($"dept" > 1)
+ .offset(1)
+ .limit(1)
+ checkLimitRemoved(df9, false)
+ checkOffsetRemoved(df9, false)
+ checkPushedInfo(df9,
+ "PushedFilters: [DEPT IS NOT NULL, DEPT > 1], PushedLimit: LIMIT 2, ReadSchema:")
+ checkAnswer(df9, Seq(Row(2, "david", 10000.00, 1300.0, true)))
+
+ val df10 = sql("SELECT dept, sum(salary) FROM h2.test.employee group by dept LIMIT 1 OFFSET 1")
+ checkLimitRemoved(df10, false)
+ checkOffsetRemoved(df10, false)
+ checkPushedInfo(df10,
+ "PushedAggregates: [SUM(SALARY)], PushedFilters: [], PushedGroupByExpressions: [DEPT], ")
+ checkAnswer(df10, Seq(Row(2, 22000.00)))
+
+ val name = udf { (x: String) => x.matches("cat|dav|amy") }
+ val sub = udf { (x: String) => x.substring(0, 3) }
+ val df11 = spark.read
+ .table("h2.test.employee")
+ .select($"SALARY", $"BONUS", sub($"NAME").as("shortName"))
+ .filter(name($"shortName"))
+ .offset(1)
+ .limit(1)
+ checkLimitRemoved(df11, false)
+ checkOffsetRemoved(df11, false)
+ checkPushedInfo(df11, "PushedFilters: [], ")
+ checkAnswer(df11, Seq(Row(9000.00, 1200.0, "cat")))
+ }
+
private def checkSortRemoved(df: DataFrame, removed: Boolean = true): Unit = {
val sorts = df.queryExecution.optimizedPlan.collect {
case s: Sort => s
@@ -225,7 +576,7 @@ class JDBCV2Suite extends QueryTest with SharedSparkSession with ExplainSuiteHel
checkSortRemoved(df1)
checkLimitRemoved(df1)
checkPushedInfo(df1,
- "PushedFilters: [], PushedTopN: ORDER BY [salary ASC NULLS FIRST] LIMIT 1, ")
+ "PushedFilters: [], PushedTopN: ORDER BY [SALARY ASC NULLS FIRST] LIMIT 1, ")
checkAnswer(df1, Seq(Row(1, "cathy", 9000.00, 1200.0, false)))
val df2 = spark.read
@@ -240,7 +591,7 @@ class JDBCV2Suite extends QueryTest with SharedSparkSession with ExplainSuiteHel
checkSortRemoved(df2)
checkLimitRemoved(df2)
checkPushedInfo(df2, "PushedFilters: [DEPT IS NOT NULL, DEPT = 1], " +
- "PushedTopN: ORDER BY [salary ASC NULLS FIRST] LIMIT 1, ")
+ "PushedTopN: ORDER BY [SALARY ASC NULLS FIRST] LIMIT 1, ")
checkAnswer(df2, Seq(Row(1, "cathy", 9000.00, 1200.0, false)))
val df3 = spark.read
@@ -255,7 +606,7 @@ class JDBCV2Suite extends QueryTest with SharedSparkSession with ExplainSuiteHel
checkSortRemoved(df3, false)
checkLimitRemoved(df3, false)
checkPushedInfo(df3, "PushedFilters: [DEPT IS NOT NULL, DEPT > 1], " +
- "PushedTopN: ORDER BY [salary DESC NULLS LAST] LIMIT 1, ")
+ "PushedTopN: ORDER BY [SALARY DESC NULLS LAST] LIMIT 1, ")
checkAnswer(df3, Seq(Row(2, "alex", 12000.00, 1200.0, false)))
val df4 =
@@ -264,7 +615,7 @@ class JDBCV2Suite extends QueryTest with SharedSparkSession with ExplainSuiteHel
checkSortRemoved(df4)
checkLimitRemoved(df4)
checkPushedInfo(df4, "PushedFilters: [DEPT IS NOT NULL, DEPT > 1], " +
- "PushedTopN: ORDER BY [salary ASC NULLS LAST] LIMIT 1, ")
+ "PushedTopN: ORDER BY [SALARY ASC NULLS LAST] LIMIT 1, ")
checkAnswer(df4, Seq(Row("david")))
val df5 = spark.read.table("h2.test.employee")
@@ -307,6 +658,38 @@ class JDBCV2Suite extends QueryTest with SharedSparkSession with ExplainSuiteHel
checkLimitRemoved(df8, false)
checkPushedInfo(df8, "PushedFilters: [], ")
checkAnswer(df8, Seq(Row(2, "alex", 12000.00, 1200.0, false)))
+
+ val df9 = spark.read
+ .table("h2.test.employee")
+ .select($"DEPT", $"name", $"SALARY",
+ when(($"SALARY" > 8000).and($"SALARY" < 10000), $"salary").otherwise(0).as("key"))
+ .sort("key", "dept", "SALARY")
+ .limit(3)
+ checkSortRemoved(df9)
+ checkLimitRemoved(df9)
+ checkPushedInfo(df9, "PushedFilters: [], " +
+ "PushedTopN: ORDER BY [CASE WHEN (SALARY > 8000.00) AND " +
+ "(SALARY < 10000.00) THEN SALARY ELSE 0.00 END ASC NULL..., ")
+ checkAnswer(df9,
+ Seq(Row(1, "amy", 10000, 0), Row(2, "david", 10000, 0), Row(2, "alex", 12000, 0)))
+
+ val df10 = spark.read
+ .option("partitionColumn", "dept")
+ .option("lowerBound", "0")
+ .option("upperBound", "2")
+ .option("numPartitions", "2")
+ .table("h2.test.employee")
+ .select($"DEPT", $"name", $"SALARY",
+ when(($"SALARY" > 8000).and($"SALARY" < 10000), $"salary").otherwise(0).as("key"))
+ .orderBy($"key", $"dept", $"SALARY")
+ .limit(3)
+ checkSortRemoved(df10, false)
+ checkLimitRemoved(df10, false)
+ checkPushedInfo(df10, "PushedFilters: [], " +
+ "PushedTopN: ORDER BY [CASE WHEN (SALARY > 8000.00) AND " +
+ "(SALARY < 10000.00) THEN SALARY ELSE 0.00 END ASC NULL..., ")
+ checkAnswer(df10,
+ Seq(Row(1, "amy", 10000, 0), Row(2, "david", 10000, 0), Row(2, "alex", 12000, 0)))
}
test("simple scan with top N: order by with alias") {
@@ -393,6 +776,94 @@ class JDBCV2Suite extends QueryTest with SharedSparkSession with ExplainSuiteHel
checkFiltersRemoved(df10)
checkPushedInfo(df10, "PushedFilters: [ID IS NOT NULL, ID > 1], ")
checkAnswer(df10, Row("mary", 2))
+
+ val df11 = sql(
+ """
+ |SELECT * FROM h2.test.employee
+ |WHERE GREATEST(bonus, 1100) > 1200 AND LEAST(salary, 10000) > 9000 AND RAND(1) < 1
+ |""".stripMargin)
+ checkFiltersRemoved(df11)
+ checkPushedInfo(df11, "PushedFilters: " +
+ "[(GREATEST(BONUS, 1100.0)) > 1200.0, (LEAST(SALARY, 10000.00)) > 9000.00, RAND(1) < 1.0]")
+ checkAnswer(df11, Row(2, "david", 10000, 1300, true))
+
+ val df12 = sql(
+ """
+ |SELECT * FROM h2.test.employee
+ |WHERE IF(SALARY > 10000, SALARY, LEAST(SALARY, 1000)) > 1200
+ |""".stripMargin)
+ checkFiltersRemoved(df12)
+ checkPushedInfo(df12, "PushedFilters: " +
+ "[(CASE WHEN SALARY > 10000.00 THEN SALARY ELSE LEAST(SALARY, 1000.00) END) > 1200.00]")
+ checkAnswer(df12, Seq(Row(2, "alex", 12000, 1200, false), Row(6, "jen", 12000, 1200, true)))
+
+ val df13 = spark.table("h2.test.employee")
+ .filter(logarithm($"bonus") > 7)
+ .filter(exp($"bonus") > 0)
+ .filter(pow($"bonus", 2) === 1440000)
+ .filter(sqrt($"bonus") > 34)
+ .filter(floor($"bonus") === 1200)
+ .filter(ceil($"bonus") === 1200)
+ checkFiltersRemoved(df13)
+ checkPushedInfo(df13, "PushedFilters: [BONUS IS NOT NULL, LN(BONUS) > 7.0, EXP(BONUS) > 0.0, " +
+ "(POWER(BONUS, 2.0)) = 1440000.0, SQRT(BONU...,")
+ checkAnswer(df13, Seq(Row(1, "cathy", 9000, 1200, false),
+ Row(2, "alex", 12000, 1200, false), Row(6, "jen", 12000, 1200, true)))
+
+ // H2 does not support width_bucket
+ val df14 = sql(
+ """
+ |SELECT * FROM h2.test.employee
+ |WHERE width_bucket(bonus, 1, 6, 3) > 4
+ |""".stripMargin)
+ checkFiltersRemoved(df14, false)
+ checkPushedInfo(df14, "PushedFilters: [BONUS IS NOT NULL]")
+ checkAnswer(df14, Seq.empty[Row])
+
+ val df15 = spark.table("h2.test.employee")
+ .filter(logarithm(2, $"bonus") > 10)
+ .filter(log10($"bonus") > 3)
+ .filter(round($"bonus") === 1200)
+ .filter(degrees($"bonus") > 68754)
+ .filter(radians($"bonus") > 20)
+ .filter(signum($"bonus") === 1)
+ checkFiltersRemoved(df15)
+ checkPushedInfo(df15, "PushedFilters: [BONUS IS NOT NULL, (LOG(2.0, BONUS)) > 10.0, " +
+ "LOG10(BONUS) > 3.0, (ROUND(BONUS, 0)) = 1200.0, DEG...,")
+ checkAnswer(df15, Seq(Row(1, "cathy", 9000, 1200, false),
+ Row(2, "alex", 12000, 1200, false), Row(6, "jen", 12000, 1200, true)))
+
+ val df16 = spark.table("h2.test.employee")
+ .filter(sin($"bonus") < -0.08)
+ .filter(sinh($"bonus") > 200)
+ .filter(cos($"bonus") > 0.9)
+ .filter(cosh($"bonus") > 200)
+ .filter(tan($"bonus") < -0.08)
+ .filter(tanh($"bonus") === 1)
+ .filter(asin($"bonus") > 0.1)
+ .filter(acos($"bonus") > 1.4)
+ .filter(atan($"bonus") > 1.4)
+ .filter(atan2($"bonus", $"bonus") > 0.7)
+ checkFiltersRemoved(df16)
+ checkPushedInfo(df16, "PushedFilters: [BONUS IS NOT NULL, SIN(BONUS) < -0.08, " +
+ "SINH(BONUS) > 200.0, COS(BONUS) > 0.9, COSH(BONUS) > 200....,")
+ checkAnswer(df16, Seq(Row(1, "cathy", 9000, 1200, false),
+ Row(2, "alex", 12000, 1200, false), Row(6, "jen", 12000, 1200, true)))
+
+ // H2 does not support log2, asinh, acosh, atanh, cbrt
+ val df17 = sql(
+ """
+ |SELECT * FROM h2.test.employee
+ |WHERE log2(dept) > 2.5
+ |AND asinh(bonus / salary) > 0.09
+ |AND acosh(dept) > 2.4
+ |AND atanh(bonus / salary) > 0.1
+ |AND cbrt(dept) > 1.8
+ |""".stripMargin)
+ checkFiltersRemoved(df17, false)
+ checkPushedInfo(df17,
+ "PushedFilters: [DEPT IS NOT NULL, BONUS IS NOT NULL, SALARY IS NOT NULL]")
+ checkAnswer(df17, Seq(Row(6, "jen", 12000, 1200, true)))
}
test("scan with filter push-down with ansi mode") {
@@ -426,10 +897,11 @@ class JDBCV2Suite extends QueryTest with SharedSparkSession with ExplainSuiteHel
checkAnswer(df2, Seq.empty)
}
- val df3 = sql("""
- |SELECT * FROM h2.test.employee
- |WHERE (CASE WHEN SALARY > 10000 THEN BONUS ELSE BONUS + 200 END) > 1200
- |""".stripMargin)
+ val df3 = sql(
+ """
+ |SELECT * FROM h2.test.employee
+ |WHERE (CASE WHEN SALARY > 10000 THEN BONUS ELSE BONUS + 200 END) > 1200
+ |""".stripMargin)
checkFiltersRemoved(df3, ansiMode)
val expectedPlanFragment3 = if (ansiMode) {
@@ -469,46 +941,22 @@ class JDBCV2Suite extends QueryTest with SharedSparkSession with ExplainSuiteHel
checkAnswer(df5, Seq(Row(1, "amy", 10000, 1000, true),
Row(1, "cathy", 9000, 1200, false), Row(6, "jen", 12000, 1200, true)))
- val df6 = spark.table("h2.test.employee")
- .filter(ln($"dept") > 1)
- .filter(exp($"salary") > 2000)
- .filter(pow($"dept", 2) > 4)
- .filter(sqrt($"salary") > 100)
- checkFiltersRemoved(df6, ansiMode)
- val expectedPlanFragment6 = if (ansiMode) {
- "PushedFilters: [DEPT IS NOT NULL, SALARY IS NOT NULL, " +
- "LN(CAST(DEPT AS double)) > 1.0, EXP(CAST(SALARY AS double)...,"
- } else {
- "PushedFilters: [DEPT IS NOT NULL, SALARY IS NOT NULL]"
- }
- checkPushedInfo(df6, expectedPlanFragment6)
- checkAnswer(df6, Seq(Row(6, "jen", 12000, 1200, true)))
-
- // H2 does not support width_bucket
- val df7 = sql("""
- |SELECT * FROM h2.test.employee
- |WHERE width_bucket(dept, 1, 6, 3) > 1
- |""".stripMargin)
- checkFiltersRemoved(df7, false)
- checkPushedInfo(df7, "PushedFilters: [DEPT IS NOT NULL]")
- checkAnswer(df7, Seq(Row(6, "jen", 12000, 1200, true)))
-
- val df8 = sql(
+ val df6 = sql(
"""
|SELECT * FROM h2.test.employee
|WHERE cast(bonus as string) like '%30%'
|AND cast(dept as byte) > 1
|AND cast(dept as short) > 1
|AND cast(bonus as decimal(20, 2)) > 1200""".stripMargin)
- checkFiltersRemoved(df8, ansiMode)
+ checkFiltersRemoved(df6, ansiMode)
val expectedPlanFragment8 = if (ansiMode) {
"PushedFilters: [BONUS IS NOT NULL, DEPT IS NOT NULL, " +
"CAST(BONUS AS string) LIKE '%30%', CAST(DEPT AS byte) > 1, ...,"
} else {
"PushedFilters: [BONUS IS NOT NULL, DEPT IS NOT NULL],"
}
- checkPushedInfo(df8, expectedPlanFragment8)
- checkAnswer(df8, Seq(Row(2, "david", 10000, 1300, true)))
+ checkPushedInfo(df6, expectedPlanFragment8)
+ checkAnswer(df6, Seq(Row(2, "david", 10000, 1300, true)))
}
}
}
@@ -1251,11 +1699,21 @@ class JDBCV2Suite extends QueryTest with SharedSparkSession with ExplainSuiteHel
}
test("column name with composite field") {
- checkAnswer(sql("SELECT dept_id FROM h2.test.dept"), Seq(Row(1), Row(2)))
- val df = sql("SELECT COUNT(dept_id) FROM h2.test.dept")
- checkAggregateRemoved(df)
- checkPushedInfo(df, "PushedAggregates: [COUNT(dept_id)]")
- checkAnswer(df, Seq(Row(2)))
+ checkAnswer(sql("SELECT `dept_id`, `dept_id1` FROM h2.test.dept"), Seq(Row(1, 1), Row(2, 1)))
+
+ val df1 = sql("SELECT COUNT(`dept_id`) FROM h2.test.dept")
+ checkPushedInfo(df1, "PushedAggregates: [COUNT(dept_id)]")
+ checkAnswer(df1, Seq(Row(2)))
+
+ val df2 = sql("SELECT `dept_id1`, COUNT(`dept_id`) FROM h2.test.dept GROUP BY `dept_id1`")
+ checkPushedInfo(df2,
+ "PushedGroupByExpressions: [dept_id1]", "PushedAggregates: [COUNT(dept_id)]")
+ checkAnswer(df2, Seq(Row(1, 2)))
+
+ val df3 = sql("SELECT `dept_id`, COUNT(`dept_id1`) FROM h2.test.dept GROUP BY `dept_id`")
+ checkPushedInfo(df3,
+ "PushedGroupByExpressions: [dept_id]", "PushedAggregates: [COUNT(dept_id1)]")
+ checkAnswer(df3, Seq(Row(1, 1), Row(2, 1)))
}
test("column name with non-ascii") {