diff --git a/assembly/pom.xml b/assembly/pom.xml index c5821de0de7cb..7e6db60169c1a 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.2.0-kylin-4.x-r66 + 3.2.0-kylin-4.x-r67 ../pom.xml diff --git a/common/kvstore/pom.xml b/common/kvstore/pom.xml index 1fc2ee62fc942..1dcd654f1efbb 100644 --- a/common/kvstore/pom.xml +++ b/common/kvstore/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.2.0-kylin-4.x-r66 + 3.2.0-kylin-4.x-r67 ../../pom.xml diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml index 2594ddbd2e758..3187d7a5fdf79 100644 --- a/common/network-common/pom.xml +++ b/common/network-common/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.2.0-kylin-4.x-r66 + 3.2.0-kylin-4.x-r67 ../../pom.xml diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml index 6a9c4d51a8441..bebd4b4fb5e1c 100644 --- a/common/network-shuffle/pom.xml +++ b/common/network-shuffle/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.2.0-kylin-4.x-r66 + 3.2.0-kylin-4.x-r67 ../../pom.xml diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml index 1507435c4f2e0..34b5e2d2bfab4 100644 --- a/common/network-yarn/pom.xml +++ b/common/network-yarn/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.2.0-kylin-4.x-r66 + 3.2.0-kylin-4.x-r67 ../../pom.xml diff --git a/common/sketch/pom.xml b/common/sketch/pom.xml index 39ea94d88d05d..ad84efbf09474 100644 --- a/common/sketch/pom.xml +++ b/common/sketch/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.2.0-kylin-4.x-r66 + 3.2.0-kylin-4.x-r67 ../../pom.xml diff --git a/common/tags/pom.xml b/common/tags/pom.xml index 872ebbb2e4309..d7a8094c2f72d 100644 --- a/common/tags/pom.xml +++ b/common/tags/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.2.0-kylin-4.x-r66 + 3.2.0-kylin-4.x-r67 ../../pom.xml diff --git a/common/unsafe/pom.xml b/common/unsafe/pom.xml index ae97b5a16798b..baaf13c497a86 100644 --- a/common/unsafe/pom.xml +++ b/common/unsafe/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.2.0-kylin-4.x-r66 + 3.2.0-kylin-4.x-r67 ../../pom.xml diff --git a/core/pom.xml b/core/pom.xml index b9bfc9f7b1efe..01ee2f723a848 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.2.0-kylin-4.x-r66 + 3.2.0-kylin-4.x-r67 ../pom.xml 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/examples/pom.xml b/examples/pom.xml index 1243b1f1cfbc6..858dd29dec8f2 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.2.0-kylin-4.x-r66 + 3.2.0-kylin-4.x-r67 ../pom.xml diff --git a/external/avro/pom.xml b/external/avro/pom.xml index cef0d3487ae35..b54fc279aa0dd 100644 --- a/external/avro/pom.xml +++ b/external/avro/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.2.0-kylin-4.x-r66 + 3.2.0-kylin-4.x-r67 ../../pom.xml diff --git a/external/docker-integration-tests/pom.xml b/external/docker-integration-tests/pom.xml index 1d41e00f57204..cbe014c6c8044 100644 --- a/external/docker-integration-tests/pom.xml +++ b/external/docker-integration-tests/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.2.0-kylin-4.x-r66 + 3.2.0-kylin-4.x-r67 ../../pom.xml diff --git a/external/kafka-0-10-assembly/pom.xml b/external/kafka-0-10-assembly/pom.xml index e15b1cf3ea14b..03c6ac670b948 100644 --- a/external/kafka-0-10-assembly/pom.xml +++ b/external/kafka-0-10-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.2.0-kylin-4.x-r66 + 3.2.0-kylin-4.x-r67 ../../pom.xml diff --git a/external/kafka-0-10-sql/pom.xml b/external/kafka-0-10-sql/pom.xml index d0017253e4d3a..fc79109166717 100644 --- a/external/kafka-0-10-sql/pom.xml +++ b/external/kafka-0-10-sql/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.2.0-kylin-4.x-r66 + 3.2.0-kylin-4.x-r67 ../../pom.xml diff --git a/external/kafka-0-10-token-provider/pom.xml b/external/kafka-0-10-token-provider/pom.xml index 749d65ac2e3e2..92de5a8199fe6 100644 --- a/external/kafka-0-10-token-provider/pom.xml +++ b/external/kafka-0-10-token-provider/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.2.0-kylin-4.x-r66 + 3.2.0-kylin-4.x-r67 ../../pom.xml diff --git a/external/kafka-0-10/pom.xml b/external/kafka-0-10/pom.xml index 00a4ff61de435..50ec3782affe8 100644 --- a/external/kafka-0-10/pom.xml +++ b/external/kafka-0-10/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.2.0-kylin-4.x-r66 + 3.2.0-kylin-4.x-r67 ../../pom.xml diff --git a/external/kinesis-asl-assembly/pom.xml b/external/kinesis-asl-assembly/pom.xml index 1144bf99d47e5..06d3d1ae09937 100644 --- a/external/kinesis-asl-assembly/pom.xml +++ b/external/kinesis-asl-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.2.0-kylin-4.x-r66 + 3.2.0-kylin-4.x-r67 ../../pom.xml diff --git a/external/kinesis-asl/pom.xml b/external/kinesis-asl/pom.xml index 7c1dd9ce0f092..c31ac2c3be08d 100644 --- a/external/kinesis-asl/pom.xml +++ b/external/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.12 - 3.2.0-kylin-4.x-r66 + 3.2.0-kylin-4.x-r67 ../../pom.xml diff --git a/external/spark-ganglia-lgpl/pom.xml b/external/spark-ganglia-lgpl/pom.xml index 1ed1d67a3c68b..54c44690ba18f 100644 --- a/external/spark-ganglia-lgpl/pom.xml +++ b/external/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.12 - 3.2.0-kylin-4.x-r66 + 3.2.0-kylin-4.x-r67 ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index e96ced2c2d44c..fd0315d50c1c3 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.2.0-kylin-4.x-r66 + 3.2.0-kylin-4.x-r67 ../pom.xml diff --git a/hadoop-cloud/pom.xml b/hadoop-cloud/pom.xml index a956f2b41d28d..0a19576190dfe 100644 --- a/hadoop-cloud/pom.xml +++ b/hadoop-cloud/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.2.0-kylin-4.x-r66 + 3.2.0-kylin-4.x-r67 ../pom.xml diff --git a/launcher/pom.xml b/launcher/pom.xml index 2f10c88f86d98..77daa6484098a 100644 --- a/launcher/pom.xml +++ b/launcher/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.2.0-kylin-4.x-r66 + 3.2.0-kylin-4.x-r67 ../pom.xml diff --git a/mllib-local/pom.xml b/mllib-local/pom.xml index d3f476b7a993d..ae37be7a7356d 100644 --- a/mllib-local/pom.xml +++ b/mllib-local/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.2.0-kylin-4.x-r66 + 3.2.0-kylin-4.x-r67 ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 1154e14ed717a..4866de74add5f 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.2.0-kylin-4.x-r66 + 3.2.0-kylin-4.x-r67 ../pom.xml diff --git a/pom.xml b/pom.xml index 46c1459157bec..53ce0c6146016 100644 --- a/pom.xml +++ b/pom.xml @@ -26,7 +26,7 @@ org.apache.spark spark-parent_2.12 - 3.2.0-kylin-4.x-r66 + 3.2.0-kylin-4.x-r67 pom Spark Project Parent POM http://spark.apache.org/ diff --git a/repl/pom.xml b/repl/pom.xml index c87e24520eb42..ee9f6a69b224d 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.2.0-kylin-4.x-r66 + 3.2.0-kylin-4.x-r67 ../pom.xml diff --git a/resource-managers/kubernetes/core/pom.xml b/resource-managers/kubernetes/core/pom.xml index 6a03fe7e26988..d51109fda47c1 100644 --- a/resource-managers/kubernetes/core/pom.xml +++ b/resource-managers/kubernetes/core/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.12 - 3.2.0-kylin-4.x-r66 + 3.2.0-kylin-4.x-r67 ../../../pom.xml diff --git a/resource-managers/kubernetes/integration-tests/pom.xml b/resource-managers/kubernetes/integration-tests/pom.xml index 20a9e7ca81f65..8b97f850a5c31 100644 --- a/resource-managers/kubernetes/integration-tests/pom.xml +++ b/resource-managers/kubernetes/integration-tests/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.12 - 3.2.0-kylin-4.x-r66 + 3.2.0-kylin-4.x-r67 ../../../pom.xml diff --git a/resource-managers/mesos/pom.xml b/resource-managers/mesos/pom.xml index 14c3470fbb401..abf9121d5923d 100644 --- a/resource-managers/mesos/pom.xml +++ b/resource-managers/mesos/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.12 - 3.2.0-kylin-4.x-r66 + 3.2.0-kylin-4.x-r67 ../../pom.xml diff --git a/resource-managers/yarn/pom.xml b/resource-managers/yarn/pom.xml index 8928a958d99ac..ee045f551d218 100644 --- a/resource-managers/yarn/pom.xml +++ b/resource-managers/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.12 - 3.2.0-kylin-4.x-r66 + 3.2.0-kylin-4.x-r67 ../../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index f0edef3964931..332c33c573f14 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.2.0-kylin-4.x-r66 + 3.2.0-kylin-4.x-r67 ../../pom.xml 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 58082d5ee09c1..a00b05e81802e 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 @@ -148,6 +148,54 @@ *
  • Since version: 3.3.0
  • * * + *
  • Name: SUBSTRING + * + *
  • + *
  • Name: UPPER + * + *
  • + *
  • Name: LOWER + * + *
  • + *
  • Name: TRANSLATE + * + *
  • + *
  • Name: TRIM + * + *
  • + *
  • Name: LTRIM + * + *
  • + *
  • Name: RTRIM + * + *
  • + *
  • Name: OVERLAY + * + *
  • * * Note: SQL semantic conforms ANSI standard, so some expressions are not supported when ANSI off, * including: add, subtract, multiply, divide, remainder, pmod. diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/expressions/aggregate/Aggregation.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/expressions/aggregate/Aggregation.java index cf7dbb2978dd7..11d9e475ca1bf 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/expressions/aggregate/Aggregation.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/expressions/aggregate/Aggregation.java @@ -20,7 +20,7 @@ import java.io.Serializable; import org.apache.spark.annotation.Evolving; -import org.apache.spark.sql.connector.expressions.NamedReference; +import org.apache.spark.sql.connector.expressions.Expression; /** * Aggregation in SQL statement. @@ -30,14 +30,14 @@ @Evolving public final class Aggregation implements Serializable { private final AggregateFunc[] aggregateExpressions; - private final NamedReference[] groupByColumns; + private final Expression[] groupByExpressions; - public Aggregation(AggregateFunc[] aggregateExpressions, NamedReference[] groupByColumns) { + public Aggregation(AggregateFunc[] aggregateExpressions, Expression[] groupByExpressions) { this.aggregateExpressions = aggregateExpressions; - this.groupByColumns = groupByColumns; + this.groupByExpressions = groupByExpressions; } public AggregateFunc[] aggregateExpressions() { return aggregateExpressions; } - public NamedReference[] groupByColumns() { return groupByColumns; } + public Expression[] groupByExpressions() { return groupByExpressions; } } 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 c9dfa2003e3c1..396b1d9cdd034 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 @@ -102,6 +102,10 @@ public String build(Expression expr) { case "FLOOR": case "CEIL": case "WIDTH_BUCKET": + case "SUBSTRING": + case "UPPER": + case "LOWER": + case "TRANSLATE": return visitSQLFunction(name, Arrays.stream(e.children()).map(c -> build(c)).toArray(String[]::new)); case "CASE_WHEN": { @@ -109,6 +113,18 @@ public String build(Expression expr) { Arrays.stream(e.children()).map(c -> build(c)).collect(Collectors.toList()); return visitCaseWhen(children.toArray(new String[e.children().length])); } + case "TRIM": + return visitTrim("BOTH", + Arrays.stream(e.children()).map(c -> build(c)).toArray(String[]::new)); + case "LTRIM": + return visitTrim("LEADING", + Arrays.stream(e.children()).map(c -> build(c)).toArray(String[]::new)); + case "RTRIM": + return visitTrim("TRAILING", + Arrays.stream(e.children()).map(c -> build(c)).toArray(String[]::new)); + case "OVERLAY": + return visitOverlay( + Arrays.stream(e.children()).map(c -> build(c)).toArray(String[]::new)); // TODO supports other expressions default: return visitUnexpectedExpr(expr); @@ -228,4 +244,23 @@ protected String visitSQLFunction(String funcName, String[] inputs) { protected String visitUnexpectedExpr(Expression expr) throws IllegalArgumentException { throw new IllegalArgumentException("Unexpected V2 expression: " + expr); } + + protected String visitOverlay(String[] inputs) { + assert(inputs.length == 3 || inputs.length == 4); + if (inputs.length == 3) { + return "OVERLAY(" + inputs[0] + " PLACING " + inputs[1] + " FROM " + inputs[2] + ")"; + } else { + return "OVERLAY(" + inputs[0] + " PLACING " + inputs[1] + " FROM " + inputs[2] + + " FOR " + inputs[3]+ ")"; + } + } + + protected String visitTrim(String direction, String[] inputs) { + assert(inputs.length == 1 || inputs.length == 2); + if (inputs.length == 1) { + return "TRIM(" + direction + " FROM " + inputs[0] + ")"; + } else { + return "TRIM(" + direction + " " + inputs[1] + " FROM " + inputs[0] + ")"; + } + } } 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..fc643e536daaa 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,9 @@ 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))) + // 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 +1787,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..45d554e3fa20f 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,17 @@ case class LocalLimit(limitExpr: Expression, child: LogicalPlan) extends OrderPr copy(child = newChild) } +object LimitAndOffset { + def unapply(p: GlobalLimit): Option[(Expression, Expression, LogicalPlan)] = { + p match { + case GlobalLimit(le1, Offset(le2, LocalLimit(le3, child))) if le1.eval().asInstanceOf[Int] + + le2.eval().asInstanceOf[Int] == le3.eval().asInstanceOf[Int] => + Some((le1, le2, 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/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala index 0c7a1030fd434..88c00c02597e7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala @@ -2371,8 +2371,4 @@ object QueryCompilationErrors { messageParameters = Array(fieldName.quoted, path.quoted), origin = context) } - - def noSuchFunctionError(database: String, funcInfo: String): Throwable = { - new AnalysisException(s"$database does not support function: $funcInfo") - } } 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/pom.xml b/sql/core/pom.xml index 61156964bc379..eef896c8e8f10 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.2.0-kylin-4.x-r66 + 3.2.0-kylin-4.x-r67 ../../pom.xml 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..fe6373319bac8 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 @@ -1955,6 +1955,16 @@ class Dataset[T] private[sql]( 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 `m` 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 b9847d48b2e17..4c7d6a9c622e9 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,7 +17,7 @@ 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, Multiply, Not, Or, Pow, Predicate, Remainder, Sqrt, StartsWith, StringPredicate, Subtract, UnaryMinus, WidthBucket} +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.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 @@ -201,6 +201,65 @@ class V2ExpressionBuilder( } else { None } + case substring: Substring => + val children = if (substring.len == Literal(Integer.MAX_VALUE)) { + Seq(substring.str, substring.pos) + } else { + substring.children + } + val childrenExpressions = children.flatMap(generateExpression(_)) + if (childrenExpressions.length == children.length) { + Some(new GeneralScalarExpression("SUBSTRING", + childrenExpressions.toArray[V2Expression])) + } else { + None + } + case Upper(child) => generateExpression(child) + .map(v => new GeneralScalarExpression("UPPER", Array[V2Expression](v))) + case Lower(child) => generateExpression(child) + .map(v => new GeneralScalarExpression("LOWER", Array[V2Expression](v))) + case translate: StringTranslate => + val childrenExpressions = translate.children.flatMap(generateExpression(_)) + if (childrenExpressions.length == translate.children.length) { + Some(new GeneralScalarExpression("TRANSLATE", + childrenExpressions.toArray[V2Expression])) + } else { + None + } + case trim: StringTrim => + val childrenExpressions = trim.children.flatMap(generateExpression(_)) + if (childrenExpressions.length == trim.children.length) { + Some(new GeneralScalarExpression("TRIM", childrenExpressions.toArray[V2Expression])) + } else { + None + } + case trim: StringTrimLeft => + val childrenExpressions = trim.children.flatMap(generateExpression(_)) + if (childrenExpressions.length == trim.children.length) { + Some(new GeneralScalarExpression("LTRIM", childrenExpressions.toArray[V2Expression])) + } else { + None + } + case trim: StringTrimRight => + val childrenExpressions = trim.children.flatMap(generateExpression(_)) + if (childrenExpressions.length == trim.children.length) { + Some(new GeneralScalarExpression("RTRIM", childrenExpressions.toArray[V2Expression])) + } else { + None + } + case overlay: Overlay => + val children = if (overlay.len == Literal(-1)) { + Seq(overlay.input, overlay.replace, overlay.pos) + } else { + overlay.children + } + val childrenExpressions = children.flatMap(generateExpression(_)) + if (childrenExpressions.length == children.length) { + Some(new GeneralScalarExpression("OVERLAY", + childrenExpressions.toArray[V2Expression])) + } else { + None + } // TODO supports other expressions 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 432775c9045ad..86edb5acbe292 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 @@ -162,7 +162,7 @@ case class RowDataSourceScanExec( "PushedFilters" -> pushedFilters) ++ pushedDownOperators.aggregation.fold(Map[String, String]()) { v => Map("PushedAggregates" -> seqToString(v.aggregateExpressions.map(_.describe())), - "PushedGroupByColumns" -> seqToString(v.groupByColumns.map(_.describe())))} ++ + "PushedGroupByExpressions" -> seqToString(v.groupByExpressions.map(_.describe())))} ++ topNOrLimitInfo ++ 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..5d387083a7f63 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 @@ -83,37 +83,52 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { 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 + TakeOrderedAndProjectExec(limit, 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 + TakeOrderedAndProjectExec(limit, order, projectList, planLater(child)) :: Nil case Limit(IntegerLiteral(limit), child) => - CollectLimitExec(limit, 0, planLater(child)) :: Nil - case GlobalLimitAndOffset( - IntegerLiteral(limit), - IntegerLiteral(offset), - Sort(order, true, child)) - if limit < conf.topKSortFallbackThreshold => - TakeOrderedAndProjectExec(limit, offset, order, child.output, planLater(child)) :: Nil - case GlobalLimitAndOffset( - IntegerLiteral(limit), - IntegerLiteral(offset), + CollectLimitExec(limit, planLater(child)) :: Nil + case LimitAndOffset(IntegerLiteral(limit), IntegerLiteral(offset), + Sort(order, true, child)) if limit + offset < conf.topKSortFallbackThreshold => + TakeOrderedAndProjectExec( + limit, order, child.output, planLater(child), offset) :: Nil + case LimitAndOffset(IntegerLiteral(limit), IntegerLiteral(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 + if limit + offset < conf.topKSortFallbackThreshold => + TakeOrderedAndProjectExec( + limit, order, projectList, planLater(child), offset) :: Nil + case LimitAndOffset(IntegerLiteral(limit), IntegerLiteral(offset), child) => + CollectLimitExec(limit, planLater(child), offset) :: Nil + case logical.Offset(IntegerLiteral(offset), child) => + CollectLimitExec(child = planLater(child), offset = offset) :: Nil case Tail(IntegerLiteral(limit), child) => CollectTailExec(limit, planLater(child)) :: Nil case other => planLater(other) :: Nil } case Limit(IntegerLiteral(limit), Sort(order, true, child)) if limit < conf.topKSortFallbackThreshold => - TakeOrderedAndProjectExec(limit, 0, order, child.output, planLater(child)) :: Nil + TakeOrderedAndProjectExec(limit, 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 - case _ => Nil + TakeOrderedAndProjectExec(limit, order, projectList, planLater(child)) :: Nil + // This is a global LIMIT and OFFSET over a logical sorting operator, + // where the sum of specified limit and specified offset is less than a heuristic threshold. + // In this case we generate a physical top-K sorting operator, passing down + // the limit and offset values to be evaluated inline during the physical + // sorting operation for greater efficiency. + case LimitAndOffset(IntegerLiteral(limit), IntegerLiteral(offset), + Sort(order, true, child)) if limit + offset < conf.topKSortFallbackThreshold => + TakeOrderedAndProjectExec( + limit, order, child.output, planLater(child), offset) :: Nil + case LimitAndOffset(IntegerLiteral(limit), IntegerLiteral(offset), + Project(projectList, Sort(order, true, child))) + if limit + offset < conf.topKSortFallbackThreshold => + TakeOrderedAndProjectExec(limit, order, projectList, planLater(child), offset) :: Nil + case LimitAndOffset(IntegerLiteral(limit), IntegerLiteral(offset), child) => + GlobalLimitAndOffsetExec(limit, offset, planLater(child)) :: Nil + case _ => + Nil } } @@ -775,8 +790,8 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { 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) => + GlobalLimitAndOffsetExec(offset = offset, child = planLater(child)) :: 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/AggregatePushDownUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/AggregatePushDownUtils.scala index 6d8cae544f23e..4de4f1fe7be62 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/AggregatePushDownUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/AggregatePushDownUtils.scala @@ -19,6 +19,8 @@ package org.apache.spark.sql.execution.datasources import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.connector.expressions.{Expression => V2Expression} +import org.apache.spark.sql.connector.expressions.FieldReference import org.apache.spark.sql.connector.expressions.aggregate.{AggregateFunc, Aggregation, Count, CountStar, Max, Min} import org.apache.spark.sql.execution.RowToColumnConverter import org.apache.spark.sql.execution.datasources.v2.V2ColumnUtils @@ -82,7 +84,7 @@ object AggregatePushDownUtils { } } - if (aggregation.groupByColumns.nonEmpty || dataFilters.nonEmpty) { + if (dataFilters.nonEmpty) { // Parquet/ORC footer has max/min/count for columns // e.g. SELECT COUNT(col1) FROM t // but footer doesn't have max/min/count for a column if max/min/count @@ -90,15 +92,27 @@ object AggregatePushDownUtils { // e.g. SELECT COUNT(col1) FROM t WHERE col2 = 8 // SELECT COUNT(col1) FROM t GROUP BY col2 // However, if the filter is on partition column, max/min/count can still be pushed down - // Todo: add support if groupby column is partition col - // (https://issues.apache.org/jira/browse/SPARK-36646) return None } - aggregation.groupByColumns.foreach { col => + + if (aggregation.groupByExpressions.nonEmpty && + partitionNames.size != aggregation.groupByExpressions.length) { + // If there are group by columns, we only push down if the group by columns are the same as + // the partition columns. In theory, if group by columns are a subset of partition columns, + // we should still be able to push down. e.g. if table t has partition columns p1, p2, and p3, + // SELECT MAX(c) FROM t GROUP BY p1, p2 should still be able to push down. However, the + // partial aggregation pushed down to data source needs to be + // SELECT p1, p2, p3, MAX(c) FROM t GROUP BY p1, p2, p3, and Spark layer + // needs to have a final aggregation such as SELECT MAX(c) FROM t GROUP BY p1, p2, then the + // pushed down query schema is different from the query schema at Spark. We will keep + // aggregate push down simple and don't handle this complicate case for now. + return None + } + aggregation.groupByExpressions.map(extractColName).foreach { colName => // don't push down if the group by columns are not the same as the partition columns (orders // doesn't matter because reorder can be done at data source layer) - if (col.fieldNames.length != 1 || !isPartitionCol(col.fieldNames.head)) return None - finalSchema = finalSchema.add(getStructFieldForCol(col.fieldNames.head)) + if (colName.isEmpty || !isPartitionCol(colName.get)) return None + finalSchema = finalSchema.add(getStructFieldForCol(colName.get)) } aggregation.aggregateExpressions.foreach { @@ -125,7 +139,8 @@ object AggregatePushDownUtils { def equivalentAggregations(a: Aggregation, b: Aggregation): Boolean = { a.aggregateExpressions.sortBy(_.hashCode()) .sameElements(b.aggregateExpressions.sortBy(_.hashCode())) && - a.groupByColumns.sortBy(_.hashCode()).sameElements(b.groupByColumns.sortBy(_.hashCode())) + a.groupByExpressions.sortBy(_.hashCode()) + .sameElements(b.groupByExpressions.sortBy(_.hashCode())) } /** @@ -145,4 +160,9 @@ object AggregatePushDownUtils { converter.convert(aggregatesAsRow, columnVectors.toArray) new ColumnarBatch(columnVectors.asInstanceOf[Array[ColumnVector]], 1) } + + private def extractColName(v2Expr: V2Expression): Option[String] = v2Expr match { + case f: FieldReference if f.fieldNames.length == 1 => Some(f.fieldNames.head) + case _ => None + } } 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 408da524cbb04..7209a61d39a31 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 @@ -753,14 +753,13 @@ object DataSourceStrategy protected[sql] def translateAggregation( aggregates: Seq[AggregateExpression], groupBy: Seq[Expression]): Option[Aggregation] = { - def columnAsString(e: Expression): Option[FieldReference] = e match { - case PushableColumnWithoutNestedColumn(name) => - Some(FieldReference(name).asInstanceOf[FieldReference]) + def translateGroupBy(e: Expression): Option[V2Expression] = e match { + case PushableExpression(expr) => Some(expr) case _ => None } val translatedAggregates = aggregates.flatMap(translateAggregate) - val translatedGroupBys = groupBy.flatMap(columnAsString) + val translatedGroupBys = groupBy.flatMap(translateGroupBy) if (translatedAggregates.length != aggregates.length || translatedGroupBys.length != groupBy.length) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcUtils.scala index 7758d6a515b51..41a68a89c4919 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcUtils.scala @@ -303,7 +303,7 @@ object OrcUtils extends Logging { partitionSchema: StructType, aggregation: Aggregation, aggSchema: StructType): InternalRow = { - require(aggregation.groupByColumns.length == 0, + require(aggregation.groupByExpressions.length == 0, s"aggregate $aggregation with group-by column shouldn't be pushed down") var columnsStatistics: OrcColumnStatistics = null try { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetUtils.scala index f3836ab8b5ae4..d999c6c1e91f1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetUtils.scala @@ -220,7 +220,7 @@ object ParquetUtils { val primitiveTypeBuilder = mutable.ArrayBuilder.make[PrimitiveType] val valuesBuilder = mutable.ArrayBuilder.make[Any] - assert(aggregation.groupByColumns.length == 0, "group by shouldn't be pushed down") + assert(aggregation.groupByExpressions.length == 0, "group by shouldn't be pushed down") aggregation.aggregateExpressions.foreach { agg => var value: Any = None var rowCount = 0L 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 cdcae15ef4e24..6291b3c8f953e 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 @@ -175,9 +175,14 @@ object V2ScanRelationPushDown extends Rule[LogicalPlan] with PredicateHelper wit // scalastyle:on val newOutput = scan.readSchema().toAttributes assert(newOutput.length == groupingExpressions.length + finalAggregates.length) - val groupAttrs = normalizedGroupingExpressions.zip(newOutput).map { - case (a: Attribute, b: Attribute) => b.withExprId(a.exprId) - case (_, b) => b + val groupByExprToOutputOrdinal = mutable.HashMap.empty[Expression, Int] + val groupAttrs = normalizedGroupingExpressions.zip(newOutput).zipWithIndex.map { + case ((a: Attribute, b: Attribute), _) => b.withExprId(a.exprId) + case ((expr, attr), ordinal) => + if (!groupByExprToOutputOrdinal.contains(expr.canonicalized)) { + groupByExprToOutputOrdinal(expr.canonicalized) = ordinal + } + attr } val aggOutput = newOutput.drop(groupAttrs.length) val output = groupAttrs ++ aggOutput @@ -188,7 +193,7 @@ object V2ScanRelationPushDown extends Rule[LogicalPlan] with PredicateHelper wit |Pushed Aggregate Functions: | ${pushedAggregates.get.aggregateExpressions.mkString(", ")} |Pushed Group by: - | ${pushedAggregates.get.groupByColumns.mkString(", ")} + | ${pushedAggregates.get.groupByExpressions.mkString(", ")} |Output: ${output.mkString(", ")} """.stripMargin) @@ -197,14 +202,15 @@ object V2ScanRelationPushDown extends Rule[LogicalPlan] with PredicateHelper wit DataSourceV2ScanRelation(sHolder.relation, wrappedScan, output) if (r.supportCompletePushDown(pushedAggregates.get)) { val projectExpressions = finalResultExpressions.map { expr => - // TODO At present, only push down group by attribute is supported. - // In future, more attribute conversion is extended here. e.g. GetStructField - expr.transform { + expr.transformDown { case agg: AggregateExpression => val ordinal = aggExprToOutputOrdinal(agg.canonicalized) val child = addCastIfNeeded(aggOutput(ordinal), agg.resultAttribute.dataType) Alias(child, agg.resultAttribute.name)(agg.resultAttribute.exprId) + case expr if groupByExprToOutputOrdinal.contains(expr.canonicalized) => + val ordinal = groupByExprToOutputOrdinal(expr.canonicalized) + addCastIfNeeded(groupAttrs(ordinal), expr.dataType) } }.asInstanceOf[Seq[NamedExpression]] Project(projectExpressions, scanRelation) @@ -247,6 +253,9 @@ object V2ScanRelationPushDown extends Rule[LogicalPlan] with PredicateHelper wit case other => other } agg.copy(aggregateFunction = aggFunction) + case expr if groupByExprToOutputOrdinal.contains(expr.canonicalized) => + val ordinal = groupByExprToOutputOrdinal(expr.canonicalized) + addCastIfNeeded(groupAttrs(ordinal), expr.dataType) } } } @@ -277,7 +286,7 @@ object V2ScanRelationPushDown extends Rule[LogicalPlan] with PredicateHelper wit private def supportPartialAggPushDown(agg: Aggregation): Boolean = { // We don't know the agg buffer of `GeneralAggregateFunc`, so can't do partial agg push down. // If `Sum`, `Count`, `Avg` with distinct, can't do partial agg push down. - agg.aggregateExpressions().exists { + agg.aggregateExpressions().isEmpty || agg.aggregateExpressions().exists { case sum: Sum => !sum.isDistinct case count: Count => !count.isDistinct case avg: Avg => !avg.isDistinct 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 0a1542a42956d..8b378d2d87c49 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 @@ -20,7 +20,7 @@ import scala.util.control.NonFatal import org.apache.spark.internal.Logging import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.connector.expressions.SortOrder +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} @@ -70,12 +70,15 @@ case class JDBCScanBuilder( private var pushedAggregateList: Array[String] = Array() - private var pushedGroupByCols: Option[Array[String]] = None + private var pushedGroupBys: Option[Array[String]] = None override def supportCompletePushDown(aggregation: Aggregation): Boolean = { - lazy val fieldNames = aggregation.groupByColumns()(0).fieldNames() + lazy val fieldNames = aggregation.groupByExpressions()(0) match { + case field: FieldReference => field.fieldNames + case _ => Array.empty[String] + } jdbcOptions.numPartitions.map(_ == 1).getOrElse(true) || - (aggregation.groupByColumns().length == 1 && fieldNames.length == 1 && + (aggregation.groupByExpressions().length == 1 && fieldNames.length == 1 && jdbcOptions.partitionColumn.exists(fieldNames(0).equalsIgnoreCase(_))) } @@ -86,20 +89,18 @@ case class JDBCScanBuilder( val compiledAggs = aggregation.aggregateExpressions.flatMap(dialect.compileAggregate) if (compiledAggs.length != aggregation.aggregateExpressions.length) return false - val groupByCols = aggregation.groupByColumns.map { col => - if (col.fieldNames.length != 1) return false - dialect.quoteIdentifier(col.fieldNames.head) - } + val compiledGroupBys = aggregation.groupByExpressions.flatMap(dialect.compileExpression) + if (compiledGroupBys.length != aggregation.groupByExpressions.length) return false // The column names here are already quoted and can be used to build sql string directly. // e.g. "DEPT","NAME",MAX("SALARY"),MIN("BONUS") => // SELECT "DEPT","NAME",MAX("SALARY"),MIN("BONUS") FROM "test"."employee" // GROUP BY "DEPT", "NAME" - val selectList = groupByCols ++ compiledAggs - val groupByClause = if (groupByCols.isEmpty) { + val selectList = compiledGroupBys ++ compiledAggs + val groupByClause = if (compiledGroupBys.isEmpty) { "" } else { - "GROUP BY " + groupByCols.mkString(",") + "GROUP BY " + compiledGroupBys.mkString(",") } val aggQuery = s"SELECT ${selectList.mkString(",")} FROM ${jdbcOptions.tableOrQuery} " + @@ -107,7 +108,7 @@ case class JDBCScanBuilder( try { finalSchema = JDBCRDD.getQueryOutputSchema(aggQuery, jdbcOptions, dialect) pushedAggregateList = selectList - pushedGroupByCols = Some(groupByCols) + pushedGroupBys = Some(compiledGroupBys) true } catch { case NonFatal(e) => @@ -173,6 +174,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, pushedGroupByCols, tableSample, pushedLimit, sortOrders) + pushedAggregateList, pushedGroupBys, tableSample, pushedLimit, sortOrders) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScan.scala index 6b9d181a7f4c0..39bc24e6a9eed 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScan.scala @@ -83,7 +83,7 @@ case class OrcScan( lazy private val (pushedAggregationsStr, pushedGroupByStr) = if (pushedAggregate.nonEmpty) { (seqToString(pushedAggregate.get.aggregateExpressions), - seqToString(pushedAggregate.get.groupByColumns)) + seqToString(pushedAggregate.get.groupByExpressions)) } else { ("[]", "[]") } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetScan.scala index b92ed82190ae8..8942cc2841a61 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetScan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetScan.scala @@ -114,7 +114,7 @@ case class ParquetScan( lazy private val (pushedAggregationsStr, pushedGroupByStr) = if (pushedAggregate.nonEmpty) { (seqToString(pushedAggregate.get.aggregateExpressions), - seqToString(pushedAggregate.get.groupByColumns)) + seqToString(pushedAggregate.get.groupByExpressions)) } else { ("[]", "[]") } 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..f79361ff1c530 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` + `offset` elements and 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. */ -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 + offset).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,15 @@ 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) { + if (offset > 0) { + childRDD.mapPartitionsInternal(_.take(limit + offset)) + } else { + childRDD.mapPartitionsInternal(_.take(limit)) + } + } else { + childRDD + } new ShuffledRowRDD( ShuffleExchangeExec.prepareShuffleDependency( locallyLimited, @@ -72,10 +96,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(_.drop(offset).take(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) } @@ -137,10 +176,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 = { @@ -189,37 +230,47 @@ case class GlobalLimitExec(limit: Int, child: SparkPlan) extends BaseLimitExec { * the child's single output partition. */ case class GlobalLimitAndOffsetExec( - limit: Int, - offset: Int, - child: SparkPlan) extends BaseLimitExec { + limit: Int = -1, + offset: Int, + child: SparkPlan) extends BaseLimitExec { + assert(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) + override def doExecute(): RDD[InternalRow] = if (limit >= 0) { + child.execute().mapPartitionsInternal(iter => iter.take(limit + offset).drop(offset)) + } else { + child.execute().mapPartitionsInternal(iter => iter.drop(offset)) } 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 + ctx.addMutableState( + CodeGenerator.JAVA_INT, skipTerm, forceInline = true, useFreshName = false) + if (limit >= 0) { + // 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, countTerm, forceInline = true, useFreshName = false) + s""" + | if ($skipTerm < $offset) { + | $skipTerm += 1; + | } else if ($countTerm < $limit) { + | $countTerm += 1; + | ${consume(ctx, input)} + | } + """.stripMargin + } else { + s""" + | if ($skipTerm < $offset) { + | $skipTerm += 1; + | } else { + | ${consume(ctx, input)} + | } + """.stripMargin + } } override protected def withNewChildInternal(newChild: SparkPlan): SparkPlan = @@ -235,10 +286,10 @@ case class GlobalLimitAndOffsetExec( */ 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 +297,12 @@ 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 data = if (offset > 0) { + child.execute().mapPartitionsInternal(_.map(_.copy())) + .takeOrdered(limit + offset)(ord).drop(offset) + } else { + child.execute().mapPartitionsInternal(_.map(_.copy())).takeOrdered(limit)(ord) + } if (projectList != child.output) { val proj = UnsafeProjection.create(projectList, child.output) data.map(r => proj(r).copy()) @@ -272,8 +328,14 @@ case class TakeOrderedAndProjectExec( val singlePartitionRDD = if (childRDD.getNumPartitions == 1) { childRDD } else { - val localTopK = childRDD.mapPartitions { iter => - Utils.takeOrdered(iter.map(_.copy()), limit)(ord) + val localTopK = if (offset > 0) { + childRDD.mapPartitionsInternal { iter => + Utils.takeOrdered(iter.map(_.copy()), limit + offset)(ord) + } + } else { + childRDD.mapPartitionsInternal { iter => + Utils.takeOrdered(iter.map(_.copy()), limit)(ord) + } } new ShuffledRowRDD( ShuffleExchangeExec.prepareShuffleDependency( @@ -284,8 +346,12 @@ case class TakeOrderedAndProjectExec( writeMetrics), readMetrics) } - singlePartitionRDD.mapPartitions { iter => - val topK = Utils.takeOrdered(iter.map(_.copy()), limit)(ord) + singlePartitionRDD.mapPartitionsInternal { iter => + val topK = if (offset > 0) { + Utils.takeOrdered(iter.map(_.copy()), limit + offset)(ord).drop(offset) + } else { + Utils.takeOrdered(iter.map(_.copy()), limit)(ord) + } if (projectList != child.output) { val proj = UnsafeProjection.create(projectList, child.output) topK.map(r => proj(r)) @@ -307,6 +373,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 6681aee778dbf..bf4484c6c0eea 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 @@ -17,42 +17,25 @@ package org.apache.spark.sql.jdbc -import java.sql.SQLException +import java.sql.{SQLException, Types} import java.util.Locale -import scala.util.control.NonFatal - import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.analysis.{NoSuchNamespaceException, NoSuchTableException, TableAlreadyExistsException} -import org.apache.spark.sql.connector.expressions.Expression import org.apache.spark.sql.connector.expressions.aggregate.{AggregateFunc, GeneralAggregateFunc} -import org.apache.spark.sql.errors.QueryCompilationErrors +import org.apache.spark.sql.execution.datasources.jdbc.JdbcUtils +import org.apache.spark.sql.types.{BooleanType, ByteType, DataType, DecimalType, ShortType, StringType} private object H2Dialect extends JdbcDialect { override def canHandle(url: String): Boolean = url.toLowerCase(Locale.ROOT).startsWith("jdbc:h2") - class H2SQLBuilder extends JDBCSQLBuilder { - override def visitSQLFunction(funcName: String, inputs: Array[String]): String = { - funcName match { - case "WIDTH_BUCKET" => - val functionInfo = super.visitSQLFunction(funcName, inputs) - throw QueryCompilationErrors.noSuchFunctionError("H2", functionInfo) - case _ => super.visitSQLFunction(funcName, inputs) - } - } - } + private val supportedFunctions = + Set("ABS", "COALESCE", "LN", "EXP", "POWER", "SQRT", "FLOOR", "CEIL", + "SUBSTRING", "UPPER", "LOWER", "TRANSLATE", "TRIM") - override def compileExpression(expr: Expression): Option[String] = { - val h2SQLBuilder = new H2SQLBuilder() - try { - Some(h2SQLBuilder.build(expr)) - } catch { - case NonFatal(e) => - logWarning("Error occurs while compiling V2 expression", e) - None - } - } + override def isSupportedFunction(funcName: String): Boolean = + supportedFunctions.contains(funcName) override def compileAggregate(aggFunction: AggregateFunc): Option[String] = { super.compileAggregate(aggFunction).orElse( @@ -90,6 +73,15 @@ private object H2Dialect extends JdbcDialect { ) } + override def getJDBCType(dt: DataType): Option[JdbcType] = dt match { + case StringType => Option(JdbcType("CLOB", Types.CLOB)) + case BooleanType => Some(JdbcType("BOOLEAN", Types.BOOLEAN)) + case ShortType | ByteType => Some(JdbcType("SMALLINT", Types.SMALLINT)) + case t: DecimalType => Some( + JdbcType(s"NUMERIC(${t.precision},${t.scale})", Types.NUMERIC)) + case _ => JdbcUtils.getCommonJDBCType(dt) + } + override def classifyException(message: String, e: Throwable): AnalysisException = { if (e.isInstanceOf[SQLException]) { // Error codes are from https://www.h2database.com/javadoc/org/h2/api/ErrorCode.html 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 397942d7837db..a046d9a8c71c7 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 @@ -240,8 +240,45 @@ abstract class JdbcDialect extends Serializable with Logging{ getJDBCType(dataType).map(_.databaseTypeDefinition).getOrElse(dataType.typeName) s"CAST($l AS $databaseTypeDefinition)" } + + override def visitSQLFunction(funcName: String, inputs: Array[String]): String = { + if (isSupportedFunction(funcName)) { + s"""$funcName(${inputs.mkString(", ")})""" + } else { + // The framework will catch the error and give up the push-down. + // Please see `JdbcDialect.compileExpression(expr: Expression)` for more details. + throw new UnsupportedOperationException( + s"${this.getClass.getSimpleName} does not support function: $funcName") + } + } + + override def visitOverlay(inputs: Array[String]): String = { + if (isSupportedFunction("OVERLAY")) { + super.visitOverlay(inputs) + } else { + throw new UnsupportedOperationException( + s"${this.getClass.getSimpleName} does not support function: OVERLAY") + } + } + + override def visitTrim(direction: String, inputs: Array[String]): String = { + if (isSupportedFunction("TRIM")) { + super.visitTrim(direction, inputs) + } else { + throw new UnsupportedOperationException( + s"${this.getClass.getSimpleName} does not support function: TRIM") + } + } } + /** + * Returns whether the database supports function. + * @param funcName Upper-cased function name + * @return True if the database supports function. + */ + @Since("3.3.0") + def isSupportedFunction(funcName: String): Boolean = false + /** * Converts V2 expression to String representing a SQL expression. * @param expr The V2 expression to be converted. 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 15938488a84b4..d100e73a4de24 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 3e98ec2e086a2..2b7ace43773b6 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 a9d3b04471403..d0259db7280d7 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 a9d3b04471403..d0259db7280d7 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 5fc9680bfe01f..1e722cf779dab 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 c591fd8cc06c1..35eaebb171a51 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 ff0bff3d5a0fd..7066bd1ed142e 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 @@ -220,7 +220,7 @@ Input [8]: [ss_item_sk#1, revenue#11, s_store_name#32, i_item_sk#34, i_item_desc (38) 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 f1977a9a0df5c..02c9fdd520c10 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 @@ -220,7 +220,7 @@ Input [9]: [s_store_name#2, ss_store_sk#4, revenue#13, i_item_desc#16, i_current (38) 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 df35346f26ffe..e1b716bd2186e 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, [CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#24 as decimal(22,6)))), DecimalType(22,6), true) 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, [CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#24 as decimal(22,6)))), DecimalType(22,6), true) 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 8b69de8e84252..fe910f9157d15 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, [CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#24 as decimal(22,6)))), DecimalType(22,6), true) 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, [CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#24 as decimal(22,6)))), DecimalType(22,6), true) 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 f5b47698e6a88..f071af103792d 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 03e922959ca66..33d072fb94143 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 5bb7b9e0814a7..9183bda292869 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 65cc46602bd47..c2fdcc7f61dbd 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 0884b8187e52e..64ee24cf9435c 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 4816a7ba95f24..306ecd52c1a3b 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 9519a0686d86f..536a1cc04222f 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 13c8607b8891e..cf4bb6501bd92 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 0ae1e4e54b578..3a62afcce3e31 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 6c74d7c015ae0..ae5cf49cbb21b 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 fb3ef22b4e9b5..d50622c2464ea 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 3bd4af148dcb9..b54c704b66c3f 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 8e153b7ae0377..638f5ec3ded62 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(CheckOverflow((promote_precis (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 54951d03cb540..371f34bc14b4b 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(CheckOverflow((promote_precis (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 f8fdae5355ff8..35b9877c4fd09 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 ba3c1ec14e96c..fdf276c01e19a 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 278ca2fc903fd..1ace9e7f294aa 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 7db6ab3797f67..f6c9b9ed7dcef 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 516553b0fb48a..6924f13d615bf 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 06720bd5836d2..a9cad5df37b9b 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 b1e9caf0da0ba..0d6b36d3b91d1 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 fae606db85edb..54be487d1ac09 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 ce94537d1cc62..0da152eaf66a8 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 18164d98bfb4a..7678a91036fd6 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 cc2cd8674c2ae..8fa5abffaa52f 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 2f19f0870bebf..b3d0081f5d22e 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 78d69659d5834..529b9c8282db5 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, [CheckOverflow((promote_precision(cast(sum_sales#22 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(22,6), true) 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, [CheckOverflow((promote_precision(cast(sum_sales#22 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(22,6), true) 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 c3623db1a40f4..4f69eb1367b8b 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, [CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(22,6), true) 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, [CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(22,6), true) 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 88727127efeaa..889ada3f2bd24 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 303efdac7c3af..399ab59cd7a71 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 50756d6d64481..0690c363a98e7 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 dc5e4f363017d..693a853440d32 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 a3fd0aebd950f..ea800b099f46a 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 73d2ce4c4bf5d..a2c5cba8b3548 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 c482af51861fe..b15ae61d824d4 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 5558a1ade4643..ed5cd21140cad 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 96ca5d4de1861..ef8d64cee2c4a 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, [CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(22,6), true) 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, [CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(22,6), true) 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 d704025e8c03b..a3b9279528ba9 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, [CheckOverflow((promote_precision(cast(sum_sales#20 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#24 as decimal(22,6)))), DecimalType(22,6), true) 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, [CheckOverflow((promote_precision(cast(sum_sales#20 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#24 as decimal(22,6)))), DecimalType(22,6), true) 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 7a62d4f3c1c68..8e969096c5239 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 e7dec9fa9a398..67f19d31e3946 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 7f5a215f09d40..7f2f33b2c1799 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 7f5a215f09d40..7f2f33b2c1799 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 115ad3660f092..9dd05765ecd2d 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 1f0ad823d7b5a..b49e25109080e 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 d6fdfd11536db..e4baf3b296016 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 @@ -235,7 +235,7 @@ Input [8]: [ss_item_sk#1, revenue#11, s_store_name#31, i_item_sk#34, i_item_desc (41) 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 5772fa657c55c..49cc9f75956a2 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 @@ -220,7 +220,7 @@ Input [9]: [s_store_name#2, ss_store_sk#4, revenue#13, i_item_desc#16, i_current (38) 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 3445c09c5e348..2c72da471f5a8 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 d12b5b2143a00..1d650dbdc7b5d 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 bc19fb0849318..d74fb5b4bfb61 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 ff7cbdd9c1820..a9efff6eba561 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 a3b9b678190dc..a00880bad3116 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 4cd1434bc1287..0d7bfa462ef4c 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 e73db6041cf1d..cfbaa2e8b48d2 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 715e056c7e999..c18e9a125335e 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 ec908ea2ded7a..83d227688cf61 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 79aac6339507d..260224e41b7f7 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 b48e703cd1025..175a1c675675f 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 375329f52bcde..8332d48905e48 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 feda27f29576b..9c798856baa66 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, [CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#24 as decimal(22,6)))), DecimalType(22,6), true) 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, [CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#24 as decimal(22,6)))), DecimalType(22,6), true) 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 c837723300e2d..4c6124960bb0d 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, [CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#24 as decimal(22,6)))), DecimalType(22,6), true) 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, [CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#24 as decimal(22,6)))), DecimalType(22,6), true) 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 e46590641bab4..01b7b7f5e20c8 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 06987b594e956..54b9ae752c7a0 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 90151fdeadc16..a37943bcca9cc 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 2060e411fab4f..33fcccae08903 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 d45fa32ab2045..40a9cea61aecc 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 5b85078bfbce7..479a27f8fee47 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 25c39bc97bc07..5c3fbb7946f1f 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 c4b064d917be4..212cb97de2873 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 bd391ab0e686e..5595e1a12b3fc 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 a62d2a27a4b4c..bd3290f8c55b4 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 c5d17373e6388..64a92b9e727bc 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 d1c22a82016ae..5ea1cda2f68d5 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 ac12fb9c9458c..0e20331e83484 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 ca404ca5a7cb8..5470bf61ac502 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 13113cef49334..4566f30b27d04 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, [CheckOverflow((promote_precision(cast(sum_sales#22 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(22,6), true) 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, [CheckOverflow((promote_precision(cast(sum_sales#22 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#26 as decimal(22,6)))), DecimalType(22,6), true) 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 7f1c103666280..21944f91237a0 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, [CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(22,6), true) 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, [CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(22,6), true) 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 f6d133ea16be2..b1b28f1a20048 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 a97cad823ae9d..1e11686ade7cc 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 6b33949833f17..d214b321a4791 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, [CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(22,6), true) 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, [CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#25 as decimal(22,6)))), DecimalType(22,6), true) 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 28dd635a8e406..65a811671c32d 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, [CheckOverflow((promote_precision(cast(sum_sales#20 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#24 as decimal(22,6)))), DecimalType(22,6), true) 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, [CheckOverflow((promote_precision(cast(sum_sales#20 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#24 as decimal(22,6)))), DecimalType(22,6), true) 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 678e08cad675c..b6a5a36a10c6c 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 5dbc33e4313d5..05636f5f44067 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 c33cbdc08e6f1..c7ccb242056f7 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 ffdbbb85d72a9..0e1ea31859b3d 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 46c6a55396584..b0ecc08ff8b25 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 fdc51c10aa028..48ab2f77ad964 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 e20b6a8b4c1c9..61c479bb02ce5 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 7999f3a271a55..94bfa14baea87 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 66e8104ea2cb0..cd66823f10e8c 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 @@ -685,7 +685,7 @@ Input [14]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact (127) 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 66e8104ea2cb0..cd66823f10e8c 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 @@ -685,7 +685,7 @@ Input [14]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact (127) 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 f7a6e5c60c4f9..4d27141fd8465 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 9dd8d6d0e5fd6..a1d99b72c8147 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 ec08022ce0bfc..b54f3fa20c63f 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 ec08022ce0bfc..b54f3fa20c63f 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 2ef8304c12a93..34777c108a268 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 2bc1c831edc1d..3e68f3fe694fc 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/jdbc/JDBCV2Suite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCV2Suite.scala index 389f8b9e417c9..219adc0ea6625 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 @@ -171,7 +171,7 @@ class JDBCV2Suite extends QueryTest with SharedSparkSession with ExplainSuiteHel .groupBy("DEPT").sum("SALARY") .limit(1) checkPushedInfo(df4, - "PushedAggregates: [SUM(SALARY)], PushedFilters: [], PushedGroupByColumns: [DEPT], ") + "PushedAggregates: [SUM(SALARY)], PushedFilters: [], PushedGroupByExpressions: [DEPT], ") checkAnswer(df4, Seq(Row(1, 19000.00))) val name = udf { (x: String) => x.matches("cat|dav|amy") } @@ -257,7 +257,7 @@ class JDBCV2Suite extends QueryTest with SharedSparkSession with ExplainSuiteHel .limit(1) checkSortRemoved(df6, false) checkPushedInfo(df6, "PushedAggregates: [SUM(SALARY)]," + - " PushedFilters: [], PushedGroupByColumns: [DEPT], ") + " PushedFilters: [], PushedGroupByExpressions: [DEPT], ") checkAnswer(df6, Seq(Row(1, 19000.00))) val name = udf { (x: String) => x.matches("cat|dav|amy") } @@ -465,6 +465,23 @@ class JDBCV2Suite extends QueryTest with SharedSparkSession with ExplainSuiteHel checkFiltersRemoved(df7, false) checkPushedInfo(df7, "PushedFilters: [DEPT IS NOT NULL]") checkAnswer(df7, Seq(Row(6, "jen", 12000, 1200, true))) + + val df8 = 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) + 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))) } } } @@ -601,6 +618,54 @@ class JDBCV2Suite extends QueryTest with SharedSparkSession with ExplainSuiteHel } } + test("scan with filter push-down with string functions") { + val df1 = sql("select * FROM h2.test.employee where " + + "substr(name, 2, 1) = 'e'" + + " AND upper(name) = 'JEN' AND lower(name) = 'jen' ") + checkFiltersRemoved(df1) + val expectedPlanFragment1 = + "PushedFilters: [NAME IS NOT NULL, (SUBSTRING(NAME, 2, 1)) = 'e', " + + "UPPER(NAME) = 'JEN', LOWER(NAME) = 'jen']" + checkPushedInfo(df1, expectedPlanFragment1) + checkAnswer(df1, Seq(Row(6, "jen", 12000, 1200, true))) + + val df2 = sql("select * FROM h2.test.employee where " + + "trim(name) = 'jen' AND trim('j', name) = 'en'" + + "AND translate(name, 'e', 1) = 'j1n'") + checkFiltersRemoved(df2) + val expectedPlanFragment2 = + "PushedFilters: [NAME IS NOT NULL, TRIM(BOTH FROM NAME) = 'jen', " + + "(TRIM(BOTH 'j' FROM NAME)) = 'en', (TRANSLATE(NA..." + checkPushedInfo(df2, expectedPlanFragment2) + checkAnswer(df2, Seq(Row(6, "jen", 12000, 1200, true))) + + val df3 = sql("select * FROM h2.test.employee where " + + "ltrim(name) = 'jen' AND ltrim('j', name) = 'en'") + checkFiltersRemoved(df3) + val expectedPlanFragment3 = + "PushedFilters: [TRIM(LEADING FROM NAME) = 'jen', " + + "(TRIM(LEADING 'j' FROM NAME)) = 'en']" + checkPushedInfo(df3, expectedPlanFragment3) + checkAnswer(df3, Seq(Row(6, "jen", 12000, 1200, true))) + + val df4 = sql("select * FROM h2.test.employee where " + + "rtrim(name) = 'jen' AND rtrim('n', name) = 'je'") + checkFiltersRemoved(df4) + val expectedPlanFragment4 = + "PushedFilters: [TRIM(TRAILING FROM NAME) = 'jen', " + + "(TRIM(TRAILING 'n' FROM NAME)) = 'je']" + checkPushedInfo(df4, expectedPlanFragment4) + checkAnswer(df4, Seq(Row(6, "jen", 12000, 1200, true))) + + // H2 does not support OVERLAY + val df5 = sql("select * FROM h2.test.employee where OVERLAY(NAME, '1', 2, 1) = 'j1n'") + checkFiltersRemoved(df5, false) + val expectedPlanFragment5 = + "PushedFilters: [NAME IS NOT NULL]" + checkPushedInfo(df5, expectedPlanFragment5) + checkAnswer(df5, Seq(Row(6, "jen", 12000, 1200, true))) + } + test("scan with aggregate push-down: MAX AVG with filter and group by") { val df = sql("select MAX(SaLaRY), AVG(BONUS) FROM h2.test.employee where dept > 0" + " group by DePt") @@ -608,7 +673,7 @@ class JDBCV2Suite extends QueryTest with SharedSparkSession with ExplainSuiteHel checkAggregateRemoved(df) checkPushedInfo(df, "PushedAggregates: [MAX(SALARY), AVG(BONUS)], " + "PushedFilters: [DEPT IS NOT NULL, DEPT > 0], " + - "PushedGroupByColumns: [DEPT], ") + "PushedGroupByExpressions: [DEPT], ") checkAnswer(df, Seq(Row(10000, 1100.0), Row(12000, 1250.0), Row(12000, 1200.0))) } @@ -629,7 +694,7 @@ class JDBCV2Suite extends QueryTest with SharedSparkSession with ExplainSuiteHel checkAggregateRemoved(df) checkPushedInfo(df, "PushedAggregates: [MAX(ID), AVG(ID)], " + "PushedFilters: [ID IS NOT NULL, ID > 0], " + - "PushedGroupByColumns: [], ") + "PushedGroupByExpressions: [], ") checkAnswer(df, Seq(Row(2, 1.5))) } @@ -670,6 +735,57 @@ class JDBCV2Suite extends QueryTest with SharedSparkSession with ExplainSuiteHel checkAnswer(df, Seq(Row(5))) } + test("scan with aggregate push-down: GROUP BY without aggregate functions") { + val df = sql("select name FROM h2.test.employee GROUP BY name") + checkAggregateRemoved(df) + checkPushedInfo(df, + "PushedAggregates: [], PushedFilters: [], PushedGroupByExpressions: [NAME],") + checkAnswer(df, Seq(Row("alex"), Row("amy"), Row("cathy"), Row("david"), Row("jen"))) + + val df2 = spark.read + .option("partitionColumn", "dept") + .option("lowerBound", "0") + .option("upperBound", "2") + .option("numPartitions", "2") + .table("h2.test.employee") + .groupBy($"name") + .agg(Map.empty[String, String]) + checkAggregateRemoved(df2, false) + checkPushedInfo(df2, + "PushedAggregates: [], PushedFilters: [], PushedGroupByExpressions: [NAME],") + checkAnswer(df2, Seq(Row("alex"), Row("amy"), Row("cathy"), Row("david"), Row("jen"))) + + val df3 = sql("SELECT CASE WHEN SALARY > 8000 AND SALARY < 10000 THEN SALARY ELSE 0 END as" + + " key FROM h2.test.employee GROUP BY key") + checkAggregateRemoved(df3) + checkPushedInfo(df3, + """ + |PushedAggregates: [], + |PushedFilters: [], + |PushedGroupByExpressions: + |[CASE WHEN (SALARY > 8000.00) AND (SALARY < 10000.00) THEN SALARY ELSE 0.00 END], + |""".stripMargin.replaceAll("\n", " ")) + checkAnswer(df3, Seq(Row(0), Row(9000))) + + val df4 = spark.read + .option("partitionColumn", "dept") + .option("lowerBound", "0") + .option("upperBound", "2") + .option("numPartitions", "2") + .table("h2.test.employee") + .groupBy(when(($"SALARY" > 8000).and($"SALARY" < 10000), $"SALARY").otherwise(0).as("key")) + .agg(Map.empty[String, String]) + checkAggregateRemoved(df4, false) + checkPushedInfo(df4, + """ + |PushedAggregates: [], + |PushedFilters: [], + |PushedGroupByExpressions: + |[CASE WHEN (SALARY > 8000.00) AND (SALARY < 10000.00) THEN SALARY ELSE 0.00 END], + |""".stripMargin.replaceAll("\n", " ")) + checkAnswer(df4, Seq(Row(0), Row(9000))) + } + test("scan with aggregate push-down: COUNT(col)") { val df = sql("select COUNT(DEPT) FROM h2.test.employee") checkAggregateRemoved(df) @@ -711,18 +827,84 @@ class JDBCV2Suite extends QueryTest with SharedSparkSession with ExplainSuiteHel } test("scan with aggregate push-down: SUM with group by") { - val df = sql("SELECT SUM(SALARY) FROM h2.test.employee GROUP BY DEPT") - checkAggregateRemoved(df) - checkPushedInfo(df, "PushedAggregates: [SUM(SALARY)], " + - "PushedFilters: [], PushedGroupByColumns: [DEPT], ") - checkAnswer(df, Seq(Row(19000), Row(22000), Row(12000))) + val df1 = sql("SELECT SUM(SALARY) FROM h2.test.employee GROUP BY DEPT") + checkAggregateRemoved(df1) + checkPushedInfo(df1, "PushedAggregates: [SUM(SALARY)], " + + "PushedFilters: [], PushedGroupByExpressions: [DEPT], ") + checkAnswer(df1, Seq(Row(19000), Row(22000), Row(12000))) + + val df2 = sql( + """ + |SELECT CASE WHEN SALARY > 8000 AND SALARY < 10000 THEN SALARY ELSE 0 END as key, + | SUM(SALARY) FROM h2.test.employee GROUP BY key""".stripMargin) + checkAggregateRemoved(df2) + checkPushedInfo(df2, + """ + |PushedAggregates: [SUM(SALARY)], + |PushedFilters: [], + |PushedGroupByExpressions: + |[CASE WHEN (SALARY > 8000.00) AND (SALARY < 10000.00) THEN SALARY ELSE 0.00 END], + |""".stripMargin.replaceAll("\n", " ")) + checkAnswer(df2, Seq(Row(0, 44000), Row(9000, 9000))) + + val df3 = spark.read + .option("partitionColumn", "dept") + .option("lowerBound", "0") + .option("upperBound", "2") + .option("numPartitions", "2") + .table("h2.test.employee") + .groupBy(when(($"SALARY" > 8000).and($"SALARY" < 10000), $"SALARY").otherwise(0).as("key")) + .agg(sum($"SALARY")) + checkAggregateRemoved(df3, false) + checkPushedInfo(df3, + """ + |PushedAggregates: [SUM(SALARY)], + |PushedFilters: [], + |PushedGroupByExpressions: + |[CASE WHEN (SALARY > 8000.00) AND (SALARY < 10000.00) THEN SALARY ELSE 0.00 END], + |""".stripMargin.replaceAll("\n", " ")) + checkAnswer(df3, Seq(Row(0, 44000), Row(9000, 9000))) + + val df4 = sql( + """ + |SELECT DEPT, CASE WHEN SALARY > 8000 AND SALARY < 10000 THEN SALARY ELSE 0 END as key, + | SUM(SALARY) FROM h2.test.employee GROUP BY DEPT, key""".stripMargin) + checkAggregateRemoved(df4) + checkPushedInfo(df4, + """ + |PushedAggregates: [SUM(SALARY)], + |PushedFilters: [], + |PushedGroupByExpressions: + |[DEPT, CASE WHEN (SALARY > 8000.00) AND (SALARY < 10000.00) THEN SALARY ELSE 0.00 END], + |""".stripMargin.replaceAll("\n", " ")) + checkAnswer(df4, Seq(Row(1, 0, 10000), Row(1, 9000, 9000), Row(2, 0, 22000), Row(6, 0, 12000))) + + val df5 = spark.read + .option("partitionColumn", "dept") + .option("lowerBound", "0") + .option("upperBound", "2") + .option("numPartitions", "2") + .table("h2.test.employee") + .groupBy($"DEPT", + when(($"SALARY" > 8000).and($"SALARY" < 10000), $"SALARY").otherwise(0) + .as("key")) + .agg(sum($"SALARY")) + checkAggregateRemoved(df5, false) + checkPushedInfo(df5, + """ + |PushedAggregates: [SUM(SALARY)], + |PushedFilters: [], + |PushedGroupByExpressions: + |[DEPT, CASE WHEN (SALARY > 8000.00) AND (SALARY < 10000.00) THEN SALARY ELSE 0.00 END], + |""".stripMargin.replaceAll("\n", " ")) + checkAnswer(df5, Seq(Row(1, 0, 10000), Row(1, 9000, 9000), Row(2, 0, 22000), Row(6, 0, 12000))) } test("scan with aggregate push-down: DISTINCT SUM with group by") { val df = sql("SELECT SUM(DISTINCT SALARY) FROM h2.test.employee GROUP BY DEPT") checkAggregateRemoved(df) checkPushedInfo(df, "PushedAggregates: [SUM(DISTINCT SALARY)], " + - "PushedFilters: [], PushedGroupByColumns: [DEPT]") + "PushedFilters: [], PushedGroupByExpressions: [DEPT]") checkAnswer(df, Seq(Row(19000), Row(22000), Row(12000))) } @@ -732,7 +914,7 @@ class JDBCV2Suite extends QueryTest with SharedSparkSession with ExplainSuiteHel checkFiltersRemoved(df) checkAggregateRemoved(df) checkPushedInfo(df, "PushedAggregates: [MAX(SALARY), MIN(BONUS)], " + - "PushedFilters: [DEPT IS NOT NULL, DEPT > 0], PushedGroupByColumns: [DEPT, NAME]") + "PushedFilters: [DEPT IS NOT NULL, DEPT > 0], PushedGroupByExpressions: [DEPT, NAME]") checkAnswer(df, Seq(Row(9000, 1200), Row(12000, 1200), Row(10000, 1300), Row(10000, 1000), Row(12000, 1200))) } @@ -746,7 +928,7 @@ class JDBCV2Suite extends QueryTest with SharedSparkSession with ExplainSuiteHel assert(filters1.isEmpty) checkAggregateRemoved(df1) checkPushedInfo(df1, "PushedAggregates: [MAX(SALARY)], " + - "PushedFilters: [DEPT IS NOT NULL, DEPT > 0], PushedGroupByColumns: [DEPT, NAME]") + "PushedFilters: [DEPT IS NOT NULL, DEPT > 0], PushedGroupByExpressions: [DEPT, NAME]") checkAnswer(df1, Seq(Row("1#amy", 10000), Row("1#cathy", 9000), Row("2#alex", 12000), Row("2#david", 10000), Row("6#jen", 12000))) @@ -758,7 +940,7 @@ class JDBCV2Suite extends QueryTest with SharedSparkSession with ExplainSuiteHel assert(filters2.isEmpty) checkAggregateRemoved(df2) checkPushedInfo(df2, "PushedAggregates: [MAX(SALARY), MIN(BONUS)], " + - "PushedFilters: [DEPT IS NOT NULL, DEPT > 0], PushedGroupByColumns: [DEPT, NAME]") + "PushedFilters: [DEPT IS NOT NULL, DEPT > 0], PushedGroupByExpressions: [DEPT, NAME]") checkAnswer(df2, Seq(Row("1#amy", 11000), Row("1#cathy", 10200), Row("2#alex", 13200), Row("2#david", 11300), Row("6#jen", 13200))) @@ -778,7 +960,7 @@ class JDBCV2Suite extends QueryTest with SharedSparkSession with ExplainSuiteHel checkFiltersRemoved(df, false) checkAggregateRemoved(df) checkPushedInfo(df, "PushedAggregates: [MAX(SALARY), MIN(BONUS)], " + - "PushedFilters: [DEPT IS NOT NULL, DEPT > 0], PushedGroupByColumns: [DEPT]") + "PushedFilters: [DEPT IS NOT NULL, DEPT > 0], PushedGroupByExpressions: [DEPT]") checkAnswer(df, Seq(Row(12000, 1200), Row(12000, 1200))) } @@ -788,7 +970,7 @@ class JDBCV2Suite extends QueryTest with SharedSparkSession with ExplainSuiteHel .min("SALARY").as("total") checkAggregateRemoved(df) checkPushedInfo(df, "PushedAggregates: [MIN(SALARY)], " + - "PushedFilters: [], PushedGroupByColumns: [DEPT]") + "PushedFilters: [], PushedGroupByExpressions: [DEPT]") checkAnswer(df, Seq(Row(1, 9000), Row(2, 10000), Row(6, 12000))) } @@ -803,7 +985,7 @@ class JDBCV2Suite extends QueryTest with SharedSparkSession with ExplainSuiteHel checkFiltersRemoved(query, false)// filter over aggregate not pushed down checkAggregateRemoved(query) checkPushedInfo(query, "PushedAggregates: [SUM(SALARY)], " + - "PushedFilters: [DEPT IS NOT NULL, DEPT > 0], PushedGroupByColumns: [DEPT]") + "PushedFilters: [DEPT IS NOT NULL, DEPT > 0], PushedGroupByExpressions: [DEPT]") checkAnswer(query, Seq(Row(6, 12000), Row(1, 19000), Row(2, 22000))) } @@ -835,7 +1017,7 @@ class JDBCV2Suite extends QueryTest with SharedSparkSession with ExplainSuiteHel checkFiltersRemoved(df) checkAggregateRemoved(df) checkPushedInfo(df, "PushedAggregates: [VAR_POP(BONUS), VAR_SAMP(BONUS)], " + - "PushedFilters: [DEPT IS NOT NULL, DEPT > 0], PushedGroupByColumns: [DEPT]") + "PushedFilters: [DEPT IS NOT NULL, DEPT > 0], PushedGroupByExpressions: [DEPT]") checkAnswer(df, Seq(Row(10000d, 20000d), Row(2500d, 5000d), Row(0d, null))) } @@ -845,7 +1027,7 @@ class JDBCV2Suite extends QueryTest with SharedSparkSession with ExplainSuiteHel checkFiltersRemoved(df) checkAggregateRemoved(df) checkPushedInfo(df, "PushedAggregates: [STDDEV_POP(BONUS), STDDEV_SAMP(BONUS)], " + - "PushedFilters: [DEPT IS NOT NULL, DEPT > 0], PushedGroupByColumns: [DEPT]") + "PushedFilters: [DEPT IS NOT NULL, DEPT > 0], PushedGroupByExpressions: [DEPT]") checkAnswer(df, Seq(Row(100d, 141.4213562373095d), Row(50d, 70.71067811865476d), Row(0d, null))) } @@ -855,7 +1037,7 @@ class JDBCV2Suite extends QueryTest with SharedSparkSession with ExplainSuiteHel checkFiltersRemoved(df) checkAggregateRemoved(df) checkPushedInfo(df, "PushedAggregates: [COVAR_POP(BONUS, BONUS), COVAR_SAMP(BONUS, BONUS)], " + - "PushedFilters: [DEPT IS NOT NULL, DEPT > 0], PushedGroupByColumns: [DEPT]") + "PushedFilters: [DEPT IS NOT NULL, DEPT > 0], PushedGroupByExpressions: [DEPT]") checkAnswer(df, Seq(Row(10000d, 20000d), Row(2500d, 5000d), Row(0d, null))) } @@ -865,7 +1047,7 @@ class JDBCV2Suite extends QueryTest with SharedSparkSession with ExplainSuiteHel checkFiltersRemoved(df) checkAggregateRemoved(df) checkPushedInfo(df, "PushedAggregates: [CORR(BONUS, BONUS)], " + - "PushedFilters: [DEPT IS NOT NULL, DEPT > 0], PushedGroupByColumns: [DEPT]") + "PushedFilters: [DEPT IS NOT NULL, DEPT > 0], PushedGroupByExpressions: [DEPT]") checkAnswer(df, Seq(Row(1d), Row(1d), Row(null))) } @@ -877,7 +1059,7 @@ class JDBCV2Suite extends QueryTest with SharedSparkSession with ExplainSuiteHel df2.queryExecution.optimizedPlan.collect { case relation: DataSourceV2ScanRelation => val expectedPlanFragment = - "PushedAggregates: [SUM(SALARY)], PushedFilters: [], PushedGroupByColumns: []" + "PushedAggregates: [SUM(SALARY)], PushedFilters: [], PushedGroupByExpressions: []" checkKeywordsExistsInExplain(df2, expectedPlanFragment) relation.scan match { case v1: V1ScanWrapper => @@ -930,7 +1112,7 @@ class JDBCV2Suite extends QueryTest with SharedSparkSession with ExplainSuiteHel "PushedAggregates: [COUNT(CASE WHEN (SALARY > 8000.00) AND (SALARY < 10000.00)" + " THEN SALARY ELSE 0.00 END), COUNT(CAS..., " + "PushedFilters: [], " + - "PushedGroupByColumns: [DEPT], ") + "PushedGroupByExpressions: [DEPT], ") checkAnswer(df, Seq(Row(1, 1, 1, 1, 1, 0d, 12000d, 0d, 12000d, 12000d, 0d, 2, 0d), Row(2, 2, 2, 2, 2, 10000d, 10000d, 10000d, 10000d, 10000d, 0d, 2, 0d), Row(2, 2, 2, 2, 2, 10000d, 12000d, 10000d, 12000d, 12000d, 0d, 3, 0d))) @@ -944,7 +1126,7 @@ class JDBCV2Suite extends QueryTest with SharedSparkSession with ExplainSuiteHel val expectedPlanFragment = if (ansiMode) { "PushedAggregates: [SUM(2147483647 + DEPT)], " + "PushedFilters: [], " + - "PushedGroupByColumns: []" + "PushedGroupByExpressions: []" } else { "PushedFilters: []" } @@ -1093,7 +1275,7 @@ class JDBCV2Suite extends QueryTest with SharedSparkSession with ExplainSuiteHel .filter($"total" > 1000) checkAggregateRemoved(df) checkPushedInfo(df, - "PushedAggregates: [SUM(SALARY)], PushedFilters: [], PushedGroupByColumns: [DEPT]") + "PushedAggregates: [SUM(SALARY)], PushedFilters: [], PushedGroupByExpressions: [DEPT]") checkAnswer(df, Seq(Row(1, 19000.00), Row(2, 22000.00), Row(6, 12000.00))) val df2 = spark.table("h2.test.employee") @@ -1103,7 +1285,7 @@ class JDBCV2Suite extends QueryTest with SharedSparkSession with ExplainSuiteHel .filter($"total" > 1000) checkAggregateRemoved(df2) checkPushedInfo(df2, - "PushedAggregates: [SUM(SALARY)], PushedFilters: [], PushedGroupByColumns: [DEPT]") + "PushedAggregates: [SUM(SALARY)], PushedFilters: [], PushedGroupByExpressions: [DEPT]") checkAnswer(df2, Seq(Row(1, 19000.00), Row(2, 22000.00), Row(6, 12000.00))) } @@ -1120,7 +1302,7 @@ class JDBCV2Suite extends QueryTest with SharedSparkSession with ExplainSuiteHel .filter($"total" > 1000) checkAggregateRemoved(df, false) checkPushedInfo(df, - "PushedAggregates: [SUM(SALARY)], PushedFilters: [], PushedGroupByColumns: [NAME]") + "PushedAggregates: [SUM(SALARY)], PushedFilters: [], PushedGroupByExpressions: [NAME]") checkAnswer(df, Seq(Row("alex", 12000.00), Row("amy", 10000.00), Row("cathy", 9000.00), Row("david", 10000.00), Row("jen", 12000.00))) @@ -1136,7 +1318,7 @@ class JDBCV2Suite extends QueryTest with SharedSparkSession with ExplainSuiteHel .filter($"total" > 1000) checkAggregateRemoved(df2, false) checkPushedInfo(df2, - "PushedAggregates: [SUM(SALARY)], PushedFilters: [], PushedGroupByColumns: [NAME]") + "PushedAggregates: [SUM(SALARY)], PushedFilters: [], PushedGroupByExpressions: [NAME]") checkAnswer(df2, Seq(Row("alex", 12000.00), Row("amy", 10000.00), Row("cathy", 9000.00), Row("david", 10000.00), Row("jen", 12000.00))) } diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index fd940c42cfe33..f2b1229d23510 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.2.0-kylin-4.x-r66 + 3.2.0-kylin-4.x-r67 ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index 5876b995eb971..bd49555d89f58 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.2.0-kylin-4.x-r66 + 3.2.0-kylin-4.x-r67 ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index d55fbe02766a8..87ac5ad121242 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.2.0-kylin-4.x-r66 + 3.2.0-kylin-4.x-r67 ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index 4913df0a21e7a..68cebcc6d5253 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.12 - 3.2.0-kylin-4.x-r66 + 3.2.0-kylin-4.x-r67 ../pom.xml