From fc63de15e44576edfd835f914501ceb929037bf5 Mon Sep 17 00:00:00 2001 From: Zhixiong Chen Date: Wed, 6 Jul 2022 10:45:14 +0800 Subject: [PATCH] [SPARK-38978][SQL] DS V2 supports push down OFFSET operator (#491) * [SPARK-28330][SQL] Support ANSI SQL: result offset clause in query expression ### What changes were proposed in this pull request? This is a ANSI SQL and feature id is `F861` ``` ::= [ ] [ ] [ ] [ ] ::= OFFSET { ROW | ROWS } ``` For example: ``` SELECT customer_name, customer_gender FROM customer_dimension WHERE occupation='Dancer' AND customer_city = 'San Francisco' ORDER BY customer_name; customer_name | customer_gender ----------------------+----------------- Amy X. Lang | Female Anna H. Li | Female Brian O. Weaver | Male Craig O. Pavlov | Male Doug Z. Goldberg | Male Harold S. Jones | Male Jack E. Perkins | Male Joseph W. Overstreet | Male Kevin . Campbell | Male Raja Y. Wilson | Male Samantha O. Brown | Female Steve H. Gauthier | Male William . Nielson | Male William Z. Roy | Male (14 rows) SELECT customer_name, customer_gender FROM customer_dimension WHERE occupation='Dancer' AND customer_city = 'San Francisco' ORDER BY customer_name OFFSET 8; customer_name | customer_gender -------------------+----------------- Kevin . Campbell | Male Raja Y. Wilson | Male Samantha O. Brown | Female Steve H. Gauthier | Male William . Nielson | Male William Z. Roy | Male (6 rows) ``` There are some mainstream database support the syntax. **Druid** https://druid.apache.org/docs/latest/querying/sql.html#offset **Kylin** http://kylin.apache.org/docs/tutorial/sql_reference.html#QUERYSYNTAX **Exasol** https://docs.exasol.com/sql/select.htm **Greenplum** http://docs.greenplum.org/6-8/ref_guide/sql_commands/SELECT.html **MySQL** https://dev.mysql.com/doc/refman/5.6/en/select.html **Monetdb** https://www.monetdb.org/Documentation/SQLreference/SQLSyntaxOverview#SELECT **PostgreSQL** https://www.postgresql.org/docs/11/queries-limit.html **Sqlite** https://www.sqlite.org/lang_select.html **Vertica** https://www.vertica.com/docs/9.2.x/HTML/Content/Authoring/SQLReferenceManual/Statements/SELECT/OFFSETClause.htm?zoom_highlight=offset The description for design: **1**. Consider `OFFSET` as the special case of `LIMIT`. For example: `SELECT * FROM a limit 10;` similar to `SELECT * FROM a limit 10 offset 0;` `SELECT * FROM a offset 10;` similar to `SELECT * FROM a limit -1 offset 10;` **2**. Because the current implement of `LIMIT` has good performance. For example: `SELECT * FROM a limit 10;` parsed to the logic plan as below: ``` GlobalLimit (limit = 10) |--LocalLimit (limit = 10) ``` and then the physical plan as below: ``` GlobalLimitExec (limit = 10) // Take the first 10 rows globally |--LocalLimitExec (limit = 10) // Take the first 10 rows locally ``` This operator reduce massive shuffle and has good performance. Sometimes, the logic plan transformed to the physical plan as: ``` CollectLimitExec (limit = 10) // Take the first 10 rows globally ``` If the SQL contains order by, such as `SELECT * FROM a order by c limit 10;`. This SQL will be transformed to the physical plan as below: ``` TakeOrderedAndProjectExec (limit = 10) // Take the first 10 rows after sort globally ``` Based on this situation, this PR produces the following operations. For example: `SELECT * FROM a limit 10 offset 10;` parsed to the logic plan as below: ``` GlobalLimit (limit = 10) |--LocalLimit (limit = 10) |--Offset (offset = 10) ``` After optimization, the above logic plan will be transformed to: ``` GlobalLimitAndOffset (limit = 10, offset = 10) // Limit clause accompanied by offset clause |--LocalLimit (limit = 20) // 10 + offset = 20 ``` and then the physical plan as below: ``` GlobalLimitAndOffsetExec (limit = 10, offset = 10) // Skip the first 10 rows and take the next 10 rows globally |--LocalLimitExec (limit = 20) // Take the first 20(limit + offset) rows locally ``` Sometimes, the logic plan transformed to the physical plan as: ``` CollectLimitExec (limit = 10, offset = 10) // Skip the first 10 rows and take the next 10 rows globally ``` If the SQL contains order by, such as `SELECT * FROM a order by c limit 10 offset 10;`. This SQL will be transformed to the physical plan as below: ``` TakeOrderedAndProjectExec (limit = 10, offset 10) // Skip the first 10 rows and take the next 10 rows after sort globally ``` **3**.In addition to the above, there is a special case that is only offset but no limit. For example: `SELECT * FROM a offset 10;` parsed to the logic plan as below: ``` Offset (offset = 10) // Only offset clause ``` If offset is very large, will generate a lot of overhead. So this PR will refuse use offset clause without limit clause, although we can parse, transform and execute it. A balanced idea is add a configuration item `spark.sql.forceUsingOffsetWithoutLimit` to force running query when user knows the offset is small enough. The default value of `spark.sql.forceUsingOffsetWithoutLimit` is false. This PR just came up with the idea so that it could be implemented at a better time in the future. Note: The origin PR to support this feature is https://github.com/apache/spark/pull/25416. Because the origin PR too old, there exists massive conflict which is hard to resolve. So I open this new PR to support this feature. ### Why are the changes needed? new feature ### Does this PR introduce any user-facing change? 'No' ### How was this patch tested? Exists and new UT Closes #35975 from beliefer/SPARK-28330. Authored-by: Jiaan Geng Signed-off-by: Wenchen Fan * [SPARK-39057][SQL] Offset could work without Limit ### What changes were proposed in this pull request? Currently, `Offset` must work with `Limit`. The behavior not allow to use offset alone and add offset API into `DataFrame`. If we use `Offset` alone, there are two situations: 1. If `Offset` is the last operator, collect the result to the driver and then drop/skip the first n (offset value) rows. Users can test or debug `Offset` in the way. 2. If `Offset` is the intermediate operator, shuffle all the result to one task and drop/skip the first n (offset value) rows and the result will be passed to the downstream operator. For example, `SELECT * FROM a offset 10; ` parsed to the logic plan as below: ``` Offset (offset = 10) // Only offset clause |--Relation ``` and then the physical plan as below: ``` CollectLimitExec(limit = -1, offset = 10) // Collect the result to the driver and skip the first 10 rows |--JDBCRelation ``` or ``` GlobalLimitAndOffsetExec(limit = -1, offset = 10) // Collect the result and skip the first 10 rows |--JDBCRelation ``` After this PR merged, users could input the SQL show below: ``` SELECT '' AS ten, unique1, unique2, stringu1 FROM onek ORDER BY unique1 OFFSET 990; ``` Note: https://github.com/apache/spark/pull/35975 supports offset clause, it create a logical node named `GlobalLimitAndOffset`. In fact, we can avoid use this node and use `Offset` instead and the latter is good with unify name. ### Why are the changes needed? Improve the implement of offset clause. ### Does this PR introduce _any_ user-facing change? 'No'. New feature. ### How was this patch tested? Exists test cases. Closes #36417 from beliefer/SPARK-28330_followup2. Authored-by: Jiaan Geng Signed-off-by: Wenchen Fan * [SPARK-39159][SQL] Add new Dataset API for Offset ### What changes were proposed in this pull request? Currently, Spark added `Offset` operator. This PR try to add `offset` API into `Dataset`. ### Why are the changes needed? `offset` API is very useful and construct test case more easily. ### Does this PR introduce _any_ user-facing change? 'No'. New feature. ### How was this patch tested? New tests. Closes #36519 from beliefer/SPARK-39159. Authored-by: Jiaan Geng Signed-off-by: Wenchen Fan * [SPARK-39180][SQL] Simplify the planning of limit and offset ### What changes were proposed in this pull request? This PR simplifies the planning of limit and offset: 1. Unify the semantics of physical plans that need to deal with limit + offset. These physical plans always do limit first, then offset. The planner rule should set limit and offset properly, for different plans, such as limit + offset and offset + limit. 2. Refactor the planner rule `SpecialLimit` to reuse the code of planning `TakeOrderedAndProjectExec`. 3. Let `GlobalLimitExec` to handle offset as well, so that we can remove `GlobalLimitAndOffsetExec`. This matches `CollectLimitExec`. ### Why are the changes needed? code simplification ### Does this PR introduce _any_ user-facing change? no ### How was this patch tested? existing tests Closes #36541 from cloud-fan/offset. Lead-authored-by: Wenchen Fan Co-authored-by: Wenchen Fan Signed-off-by: Wenchen Fan * [SPARK-39037][SQL] DS V2 aggregate push-down supports order by expressions ### What changes were proposed in this pull request? Currently, Spark DS V2 aggregate push-down only supports order by column. But the SQL show below is very useful and common. ``` SELECT CASE WHEN 'SALARY' > 8000.00 AND 'SALARY' < 10000.00 THEN 'SALARY' ELSE 0.00 END AS key, dept, name FROM "test"."employee" ORDER BY key ``` ### Why are the changes needed? Let DS V2 aggregate push-down supports order by expressions ### Does this PR introduce _any_ user-facing change? 'No'. New feature. ### How was this patch tested? New tests Closes #36370 from beliefer/SPARK-39037. Authored-by: Jiaan Geng Signed-off-by: Wenchen Fan * [SPARK-38978][SQL] DS V2 supports push down OFFSET operator ### What changes were proposed in this pull request? Currently, DS V2 push-down supports `LIMIT` but `OFFSET`. If we can pushing down `OFFSET` to JDBC data source, it will be better performance. ### Why are the changes needed? push down `OFFSET` could improves the performance. ### Does this PR introduce _any_ user-facing change? 'No'. New feature. ### How was this patch tested? New tests. Closes #36295 from beliefer/SPARK-38978. Authored-by: Jiaan Geng Signed-off-by: Wenchen Fan * fix ut * [SPARK-39340][SQL] DS v2 agg pushdown should allow dots in the name of top-level columns ### What changes were proposed in this pull request? It turns out that I was wrong in https://github.com/apache/spark/pull/36727 . We still have the limitation (column name cannot contain dot) in master and 3.3 braches, in a very implicit way: The `V2ExpressionBuilder` has a boolean flag `nestedPredicatePushdownEnabled` whose default value is false. When it's false, it uses `PushableColumnWithoutNestedColumn` to match columns, which doesn't support dot in names. `V2ExpressionBuilder` is only used in 2 places: 1. `PushableExpression`. This is a pattern match that is only used in v2 agg pushdown 2. `PushablePredicate`. This is a pattern match that is used in various places, but all the caller sides set `nestedPredicatePushdownEnabled` to true. This PR removes the `nestedPredicatePushdownEnabled` flag from `V2ExpressionBuilder`, and makes it always support nested fields. `PushablePredicate` is also updated accordingly to remove the boolean flag, as it's always true. ### Why are the changes needed? Fix a mistake to eliminate an unexpected limitation in DS v2 pushdown. ### Does this PR introduce _any_ user-facing change? No for end users. For data source developers, they can trigger agg pushdowm more often. ### How was this patch tested? a new test Closes #36945 from cloud-fan/dsv2. Authored-by: Wenchen Fan Signed-off-by: Wenchen Fan * [SPARK-39453][SQL] DS V2 supports push down misc non-aggregate functions(non ANSI) ### What changes were proposed in this pull request? https://github.com/apache/spark/pull/36039 makes DS V2 supports push down misc non-aggregate functions are claimed by ANSI standard. Spark have a lot common used misc non-aggregate functions are not claimed by ANSI standard. https://github.com/apache/spark/blob/2f8613f22c0750c00cf1dcfb2f31c431d8dc1be7/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala#L362. The mainstream databases support these functions show below. | Function name | PostgreSQL | ClickHouse | H2 | MySQL | Oracle | Redshift | Presto | Teradata | Snowflake | DB2 | Vertica | Exasol | SqlServer | Yellowbrick | Impala | Mariadb | Druid | Pig | Singlestore | ElasticSearch | SQLite | Influxdata | Sybase | | ---- | ---- | ---- | ---- | ---- | ---- | ---- | ---- | ---- | ---- | ---- | ---- | ---- | ---- | ---- | ---- | ---- | ---- | ---- | ---- | ---- | ---- | ---- | ---- | | `GREATEST` | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | No | Yes | Yes | No | No | No | | `LEAST` | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | No | Yes | Yes | No | No | No | | `IF` | No | Yes | No | Yes | No | No | Yes | No | Yes | No | No | Yes | No | Yes | Yes | Yes | No | No | Yes | Yes | Yes | No | No | | `RAND` | No | Yes | Yes | Yes | No | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | No | Yes | Yes | Yes | Yes | No | Yes | ### Why are the changes needed? DS V2 supports push down misc non-aggregate functions supported by mainstream databases. ### Does this PR introduce _any_ user-facing change? 'No'. New feature. ### How was this patch tested? New tests. Closes #36830 from beliefer/SPARK-38761_followup. Authored-by: Jiaan Geng Signed-off-by: Wenchen Fan * [SPARK-39479][SQL] DS V2 supports push down math functions(non ANSI) ### What changes were proposed in this pull request? https://github.com/apache/spark/pull/36140 makes DS V2 supports push down math functions are claimed by ANSI standard. Spark have a lot common used math functions are not claimed by ANSI standard. https://github.com/apache/spark/blob/2f8613f22c0750c00cf1dcfb2f31c431d8dc1be7/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala#L388 The mainstream databases support these functions show below. | Function name | PostgreSQL | ClickHouse | H2 | MySQL | Oracle | Redshift | Presto | Teradata | Snowflake | DB2 | Vertica | Exasol | SqlServer | Yellowbrick | Impala | Mariadb | Druid | Pig | Singlestore | ElasticSearch | SQLite | Influxdata | Sybase | | ---- | ---- | ---- | ---- | ---- | ---- | ---- | ---- | ---- | ---- | ---- | ---- | ---- | ---- | ---- | ---- | ---- | ---- | ---- | ---- | ---- | ---- | ---- | ---- | | `SIN` | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | | `SINH` | Yes | Yes | Yes | No | Yes | No | Yes | Yes | Yes | Yes | Yes | Yes | No | No | Yes | No | No | Yes | No | Yes | Yes | Yes | No | | `COS` | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | | `COSH` | Yes | Yes | Yes | No | Yes | No | Yes | Yes | Yes | Yes | Yes | Yes | No | No | Yes | No | No | Yes | No | Yes | Yes | Yes | No | | `TAN` | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | No | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | | `TANH` | Yes | No | Yes | No | Yes | No | Yes | Yes | Yes | Yes | Yes | Yes | No | No | Yes | No | No | Yes | No | No | No | Yes | No | | `COT` | Yes | No | Yes | Yes | No | Yes | No | No | Yes | Yes | Yes | Yes | Yes | No | Yes | Yes | Yes | No | Yes | Yes | No | No | Yes | | `ASIN` | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | | `ASINH` | Yes | Yes | No | No | No | No | No | Yes | Yes | No | No | No | No | No | No | No | No | No | No | No | Yes | Yes | No | | `ACOS` | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | | `ACOSH` | Yes | Yes | No | No | No | No | No | Yes | Yes | No | No | No | No | No | No | No | No | No | No | No | Yes | Yes | No | | `ATAN` | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | No | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | | `ATAN2` | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | No | Yes | Yes | Yes | No | Yes | Yes | Yes | Yes | Yes | | `ATANH` | Yes | Yes | No | No | No | No | No | Yes | Yes | Yes | No | No | No | No | No | No | No | No | No | No | Yes | Yes | No | | `LOG` | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | | `LOG10` | Yes | Yes | Yes | Yes | No | No | Yes | Yes | No | Yes | Yes | Yes | Yes | No | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | | `LOG2` | No | Yes | No | Yes | No | No | Yes | Yes | No | No | No | Yes | No | No | Yes | Yes | No | No | Yes | No | Yes | Yes | No | | `CBRT` | Yes | Yes | No | No | No | Yes | Yes | No | Yes | No | Yes | No | No | Yes | No | No | No | Yes | No | Yes | No | Yes | No | | `DEGREES` | Yes | Yes | Yes | Yes | No | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | No | Yes | Yes | Yes | No | Yes | | `RADIANS` | Yes | Yes | Yes | Yes | No | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | No | Yes | Yes | Yes | No | Yes | | `ROUND` | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | No | Yes | Yes | | `SIGN` | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | Yes | No | No | Yes | Yes | No | No | Yes | ### Why are the changes needed? DS V2 supports push down math functions supported by mainstream databases. ### Does this PR introduce _any_ user-facing change? 'No'. New feature. ### How was this patch tested? New tests. Closes #36877 from beliefer/SPARK-39479. Authored-by: Jiaan Geng Signed-off-by: Wenchen Fan Co-authored-by: Jiaan Geng Co-authored-by: Wenchen Fan Co-authored-by: Wenchen Fan --- docs/sql-ref-ansi-compliance.md | 2 +- .../expressions/GeneralScalarExpression.java | 150 +++++ .../spark/sql/connector/read/ScanBuilder.java | 3 +- .../connector/read/SupportsPushDownLimit.java | 4 +- .../read/SupportsPushDownOffset.java | 36 ++ .../connector/read/SupportsPushDownTopN.java | 23 +- .../util/V2ExpressionSQLBuilder.java | 25 + .../sql/catalyst/analysis/CheckAnalysis.scala | 28 +- .../sql/catalyst/optimizer/Optimizer.scala | 33 +- .../optimizer/PropagateEmptyRelation.scala | 2 +- .../sql/catalyst/optimizer/expressions.scala | 1 + .../plans/logical/basicLogicalOperators.scala | 45 +- .../SizeInBytesOnlyStatsPlanVisitor.scala | 2 +- .../analysis/AnalysisErrorSuite.scala | 33 ++ .../optimizer/EliminateOffsetsSuite.scala | 100 ++++ .../optimizer/LimitPushdownSuite.scala | 6 + .../BasicStatsEstimationSuite.scala | 16 + .../scala/org/apache/spark/sql/Dataset.scala | 11 +- .../catalyst/util/V2ExpressionBuilder.scala | 115 +++- .../sql/execution/DataSourceScanExec.scala | 9 +- .../spark/sql/execution/SparkStrategies.scala | 81 ++- .../datasources/DataSourceStrategy.scala | 14 +- .../datasources/jdbc/JDBCOptions.scala | 5 + .../execution/datasources/jdbc/JDBCRDD.scala | 15 +- .../datasources/jdbc/JDBCRelation.scala | 7 +- .../datasources/v2/DataSourceV2Strategy.scala | 38 +- .../datasources/v2/PushDownUtils.scala | 17 +- .../datasources/v2/PushedDownOperators.scala | 1 + .../v2/V2ScanRelationPushDown.scala | 93 ++- .../datasources/v2/jdbc/JDBCScan.scala | 6 +- .../datasources/v2/jdbc/JDBCScanBuilder.scala | 30 +- .../apache/spark/sql/execution/limit.scala | 164 ++++-- .../org/apache/spark/sql/jdbc/H2Dialect.scala | 6 +- .../apache/spark/sql/jdbc/JdbcDialects.scala | 7 + .../sql-tests/inputs/postgreSQL/limit.sql | 36 +- .../exists-subquery/exists-orderby-limit.sql | 78 +++ .../inputs/subquery/in-subquery/in-limit.sql | 111 +++- .../results/postgreSQL/limit.sql.out | 86 ++- .../exists-orderby-limit.sql.out | 176 +++++- .../subquery/in-subquery/in-limit.sql.out | 174 +++++- .../q10.sf100/explain.txt | 2 +- .../approved-plans-modified/q10/explain.txt | 2 +- .../q19.sf100/explain.txt | 2 +- .../approved-plans-modified/q19/explain.txt | 2 +- .../q27.sf100/explain.txt | 2 +- .../approved-plans-modified/q27/explain.txt | 2 +- .../q3.sf100/explain.txt | 2 +- .../approved-plans-modified/q3/explain.txt | 2 +- .../q42.sf100/explain.txt | 2 +- .../approved-plans-modified/q42/explain.txt | 2 +- .../q43.sf100/explain.txt | 2 +- .../approved-plans-modified/q43/explain.txt | 2 +- .../q46.sf100/explain.txt | 2 +- .../approved-plans-modified/q46/explain.txt | 2 +- .../q52.sf100/explain.txt | 2 +- .../approved-plans-modified/q52/explain.txt | 2 +- .../q53.sf100/explain.txt | 2 +- .../approved-plans-modified/q53/explain.txt | 2 +- .../q55.sf100/explain.txt | 2 +- .../approved-plans-modified/q55/explain.txt | 2 +- .../q59.sf100/explain.txt | 2 +- .../approved-plans-modified/q59/explain.txt | 2 +- .../q63.sf100/explain.txt | 2 +- .../approved-plans-modified/q63/explain.txt | 2 +- .../q65.sf100/explain.txt | 2 +- .../approved-plans-modified/q65/explain.txt | 2 +- .../q68.sf100/explain.txt | 2 +- .../approved-plans-modified/q68/explain.txt | 2 +- .../q7.sf100/explain.txt | 2 +- .../approved-plans-modified/q7/explain.txt | 2 +- .../q79.sf100/explain.txt | 2 +- .../approved-plans-modified/q79/explain.txt | 2 +- .../q89.sf100/explain.txt | 2 +- .../approved-plans-modified/q89/explain.txt | 2 +- .../approved-plans-v1_4/q1.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q1/explain.txt | 2 +- .../approved-plans-v1_4/q10.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q10/explain.txt | 2 +- .../approved-plans-v1_4/q11.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q11/explain.txt | 2 +- .../approved-plans-v1_4/q12.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q12/explain.txt | 2 +- .../q14a.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q14a/explain.txt | 2 +- .../q14b.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q14b/explain.txt | 2 +- .../approved-plans-v1_4/q15.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q15/explain.txt | 2 +- .../approved-plans-v1_4/q17.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q17/explain.txt | 2 +- .../approved-plans-v1_4/q18.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q18/explain.txt | 2 +- .../approved-plans-v1_4/q19.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q19/explain.txt | 2 +- .../approved-plans-v1_4/q20.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q20/explain.txt | 2 +- .../approved-plans-v1_4/q21.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q21/explain.txt | 2 +- .../approved-plans-v1_4/q22.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q22/explain.txt | 2 +- .../q23b.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q23b/explain.txt | 2 +- .../approved-plans-v1_4/q25.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q25/explain.txt | 2 +- .../approved-plans-v1_4/q26.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q26/explain.txt | 2 +- .../approved-plans-v1_4/q27.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q27/explain.txt | 2 +- .../approved-plans-v1_4/q29.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q29/explain.txt | 2 +- .../approved-plans-v1_4/q3.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q3/explain.txt | 2 +- .../approved-plans-v1_4/q30.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q30/explain.txt | 2 +- .../approved-plans-v1_4/q32.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q32/explain.txt | 2 +- .../approved-plans-v1_4/q33.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q33/explain.txt | 2 +- .../approved-plans-v1_4/q35.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q35/explain.txt | 2 +- .../approved-plans-v1_4/q36.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q36/explain.txt | 2 +- .../approved-plans-v1_4/q37.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q37/explain.txt | 2 +- .../approved-plans-v1_4/q4.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q4/explain.txt | 2 +- .../approved-plans-v1_4/q40.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q40/explain.txt | 2 +- .../approved-plans-v1_4/q41.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q41/explain.txt | 2 +- .../approved-plans-v1_4/q42.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q42/explain.txt | 2 +- .../approved-plans-v1_4/q43.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q43/explain.txt | 2 +- .../approved-plans-v1_4/q44.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q44/explain.txt | 2 +- .../approved-plans-v1_4/q45.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q45/explain.txt | 2 +- .../approved-plans-v1_4/q46.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q46/explain.txt | 2 +- .../approved-plans-v1_4/q47.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q47/explain.txt | 2 +- .../approved-plans-v1_4/q49.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q49/explain.txt | 2 +- .../approved-plans-v1_4/q5.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q5/explain.txt | 2 +- .../approved-plans-v1_4/q50.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q50/explain.txt | 2 +- .../approved-plans-v1_4/q51.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q51/explain.txt | 2 +- .../approved-plans-v1_4/q52.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q52/explain.txt | 2 +- .../approved-plans-v1_4/q53.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q53/explain.txt | 2 +- .../approved-plans-v1_4/q54.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q54/explain.txt | 2 +- .../approved-plans-v1_4/q55.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q55/explain.txt | 2 +- .../approved-plans-v1_4/q56.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q56/explain.txt | 2 +- .../approved-plans-v1_4/q57.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q57/explain.txt | 2 +- .../approved-plans-v1_4/q58.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q58/explain.txt | 2 +- .../approved-plans-v1_4/q59.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q59/explain.txt | 2 +- .../approved-plans-v1_4/q60.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q60/explain.txt | 2 +- .../approved-plans-v1_4/q62.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q62/explain.txt | 2 +- .../approved-plans-v1_4/q63.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q63/explain.txt | 2 +- .../approved-plans-v1_4/q65.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q65/explain.txt | 2 +- .../approved-plans-v1_4/q66.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q66/explain.txt | 2 +- .../approved-plans-v1_4/q67.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q67/explain.txt | 2 +- .../approved-plans-v1_4/q68.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q68/explain.txt | 2 +- .../approved-plans-v1_4/q69.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q69/explain.txt | 2 +- .../approved-plans-v1_4/q7.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q7/explain.txt | 2 +- .../approved-plans-v1_4/q70.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q70/explain.txt | 2 +- .../approved-plans-v1_4/q72.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q72/explain.txt | 2 +- .../approved-plans-v1_4/q76.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q76/explain.txt | 2 +- .../approved-plans-v1_4/q77.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q77/explain.txt | 2 +- .../approved-plans-v1_4/q79.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q79/explain.txt | 2 +- .../approved-plans-v1_4/q8.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q8/explain.txt | 2 +- .../approved-plans-v1_4/q80.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q80/explain.txt | 2 +- .../approved-plans-v1_4/q81.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q81/explain.txt | 2 +- .../approved-plans-v1_4/q82.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q82/explain.txt | 2 +- .../approved-plans-v1_4/q83.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q83/explain.txt | 2 +- .../approved-plans-v1_4/q84.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q84/explain.txt | 2 +- .../approved-plans-v1_4/q85.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q85/explain.txt | 2 +- .../approved-plans-v1_4/q86.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q86/explain.txt | 2 +- .../approved-plans-v1_4/q89.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q89/explain.txt | 2 +- .../approved-plans-v1_4/q93.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q93/explain.txt | 2 +- .../approved-plans-v1_4/q99.sf100/explain.txt | 2 +- .../approved-plans-v1_4/q99/explain.txt | 2 +- .../q10a.sf100/explain.txt | 2 +- .../approved-plans-v2_7/q10a/explain.txt | 2 +- .../approved-plans-v2_7/q11.sf100/explain.txt | 2 +- .../approved-plans-v2_7/q11/explain.txt | 2 +- .../approved-plans-v2_7/q12.sf100/explain.txt | 2 +- .../approved-plans-v2_7/q12/explain.txt | 2 +- .../approved-plans-v2_7/q14.sf100/explain.txt | 2 +- .../approved-plans-v2_7/q14/explain.txt | 2 +- .../q14a.sf100/explain.txt | 2 +- .../approved-plans-v2_7/q14a/explain.txt | 2 +- .../q18a.sf100/explain.txt | 2 +- .../approved-plans-v2_7/q18a/explain.txt | 2 +- .../approved-plans-v2_7/q20.sf100/explain.txt | 2 +- .../approved-plans-v2_7/q20/explain.txt | 2 +- .../approved-plans-v2_7/q22.sf100/explain.txt | 2 +- .../approved-plans-v2_7/q22/explain.txt | 2 +- .../q22a.sf100/explain.txt | 2 +- .../approved-plans-v2_7/q22a/explain.txt | 2 +- .../q27a.sf100/explain.txt | 2 +- .../approved-plans-v2_7/q27a/explain.txt | 2 +- .../approved-plans-v2_7/q35.sf100/explain.txt | 2 +- .../approved-plans-v2_7/q35/explain.txt | 2 +- .../q35a.sf100/explain.txt | 2 +- .../approved-plans-v2_7/q35a/explain.txt | 2 +- .../q36a.sf100/explain.txt | 2 +- .../approved-plans-v2_7/q36a/explain.txt | 2 +- .../approved-plans-v2_7/q47.sf100/explain.txt | 2 +- .../approved-plans-v2_7/q47/explain.txt | 2 +- .../approved-plans-v2_7/q49.sf100/explain.txt | 2 +- .../approved-plans-v2_7/q49/explain.txt | 2 +- .../q51a.sf100/explain.txt | 2 +- .../approved-plans-v2_7/q51a/explain.txt | 2 +- .../approved-plans-v2_7/q57.sf100/explain.txt | 2 +- .../approved-plans-v2_7/q57/explain.txt | 2 +- .../approved-plans-v2_7/q5a.sf100/explain.txt | 2 +- .../approved-plans-v2_7/q5a/explain.txt | 2 +- .../approved-plans-v2_7/q6.sf100/explain.txt | 2 +- .../approved-plans-v2_7/q6/explain.txt | 2 +- .../q67a.sf100/explain.txt | 2 +- .../approved-plans-v2_7/q67a/explain.txt | 2 +- .../q70a.sf100/explain.txt | 2 +- .../approved-plans-v2_7/q70a/explain.txt | 2 +- .../approved-plans-v2_7/q72.sf100/explain.txt | 2 +- .../approved-plans-v2_7/q72/explain.txt | 2 +- .../approved-plans-v2_7/q74.sf100/explain.txt | 2 +- .../approved-plans-v2_7/q74/explain.txt | 2 +- .../approved-plans-v2_7/q75.sf100/explain.txt | 2 +- .../approved-plans-v2_7/q75/explain.txt | 2 +- .../q77a.sf100/explain.txt | 2 +- .../approved-plans-v2_7/q77a/explain.txt | 2 +- .../approved-plans-v2_7/q78.sf100/explain.txt | 2 +- .../approved-plans-v2_7/q78/explain.txt | 2 +- .../q80a.sf100/explain.txt | 2 +- .../approved-plans-v2_7/q80a/explain.txt | 2 +- .../q86a.sf100/explain.txt | 2 +- .../approved-plans-v2_7/q86a/explain.txt | 2 +- .../org/apache/spark/sql/DataFrameSuite.scala | 24 + .../TakeOrderedAndProjectSuite.scala | 4 +- .../v2/DataSourceV2StrategySuite.scala | 2 +- .../apache/spark/sql/jdbc/JDBCV2Suite.scala | 554 ++++++++++++++++-- 276 files changed, 2293 insertions(+), 544 deletions(-) create mode 100644 sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/SupportsPushDownOffset.java create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/EliminateOffsetsSuite.scala diff --git a/docs/sql-ref-ansi-compliance.md b/docs/sql-ref-ansi-compliance.md index 7d11e93e8680c..67e138fa2e9c2 100644 --- a/docs/sql-ref-ansi-compliance.md +++ b/docs/sql-ref-ansi-compliance.md @@ -452,7 +452,7 @@ Below is a list of all the keywords in Spark SQL. |NULL|reserved|non-reserved|reserved| |NULLS|non-reserved|non-reserved|non-reserved| |OF|non-reserved|non-reserved|reserved| -|OFFSET|non-reserved|non-reserved|reserved| +|OFFSET|reserved|non-reserved|reserved| |ON|reserved|strict-non-reserved|reserved| |ONLY|reserved|non-reserved|reserved| |OPTION|non-reserved|non-reserved|non-reserved| diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/expressions/GeneralScalarExpression.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/expressions/GeneralScalarExpression.java index a00b05e81802e..134c2f36b510c 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/expressions/GeneralScalarExpression.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/expressions/GeneralScalarExpression.java @@ -106,6 +106,42 @@ *
  • Since version: 3.3.0
  • * * + *
  • Name: GREATEST + *
      + *
    • SQL semantic: GREATEST(expr, ...)
    • + *
    • Since version: 3.4.0
    • + *
    + *
  • + *
  • Name: LEAST + *
      + *
    • SQL semantic: LEAST(expr, ...)
    • + *
    • Since version: 3.4.0
    • + *
    + *
  • + *
  • Name: RAND + *
      + *
    • SQL semantic: RAND([seed])
    • + *
    • Since version: 3.4.0
    • + *
    + *
  • + *
  • Name: LOG + *
      + *
    • SQL semantic: LOG(base, expr)
    • + *
    • Since version: 3.4.0
    • + *
    + *
  • + *
  • Name: LOG10 + *
      + *
    • SQL semantic: LOG10(expr)
    • + *
    • Since version: 3.4.0
    • + *
    + *
  • + *
  • Name: LOG2 + *
      + *
    • SQL semantic: LOG2(expr)
    • + *
    • Since version: 3.4.0
    • + *
    + *
  • *
  • Name: LN *
      *
    • SQL semantic: LN(expr)
    • @@ -142,6 +178,120 @@ *
    • Since version: 3.3.0
    • *
    *
  • + *
  • Name: ROUND + *
      + *
    • SQL semantic: ROUND(expr, [scale])
    • + *
    • Since version: 3.4.0
    • + *
    + *
  • + *
  • Name: SIN + *
      + *
    • SQL semantic: SIN(expr)
    • + *
    • Since version: 3.4.0
    • + *
    + *
  • + *
  • Name: SINH + *
      + *
    • SQL semantic: SINH(expr)
    • + *
    • Since version: 3.4.0
    • + *
    + *
  • + *
  • Name: COS + *
      + *
    • SQL semantic: COS(expr)
    • + *
    • Since version: 3.4.0
    • + *
    + *
  • + *
  • Name: COSH + *
      + *
    • SQL semantic: COSH(expr)
    • + *
    • Since version: 3.4.0
    • + *
    + *
  • + *
  • Name: TAN + *
      + *
    • SQL semantic: TAN(expr)
    • + *
    • Since version: 3.4.0
    • + *
    + *
  • + *
  • Name: TANH + *
      + *
    • SQL semantic: TANH(expr)
    • + *
    • Since version: 3.4.0
    • + *
    + *
  • + *
  • Name: COT + *
      + *
    • SQL semantic: COT(expr)
    • + *
    • Since version: 3.4.0
    • + *
    + *
  • + *
  • Name: ASIN + *
      + *
    • SQL semantic: ASIN(expr)
    • + *
    • Since version: 3.4.0
    • + *
    + *
  • + *
  • Name: ASINH + *
      + *
    • SQL semantic: ASINH(expr)
    • + *
    • Since version: 3.4.0
    • + *
    + *
  • + *
  • Name: ACOS + *
      + *
    • SQL semantic: ACOS(expr)
    • + *
    • Since version: 3.4.0
    • + *
    + *
  • + *
  • Name: ACOSH + *
      + *
    • SQL semantic: ACOSH(expr)
    • + *
    • Since version: 3.4.0
    • + *
    + *
  • + *
  • Name: ATAN + *
      + *
    • SQL semantic: ATAN(expr)
    • + *
    • Since version: 3.4.0
    • + *
    + *
  • + *
  • Name: ATANH + *
      + *
    • SQL semantic: ATANH(expr)
    • + *
    • Since version: 3.4.0
    • + *
    + *
  • + *
  • Name: ATAN2 + *
      + *
    • SQL semantic: ATAN2(exprY, exprX)
    • + *
    • Since version: 3.4.0
    • + *
    + *
  • + *
  • Name: CBRT + *
      + *
    • SQL semantic: CBRT(expr)
    • + *
    • Since version: 3.4.0
    • + *
    + *
  • + *
  • Name: DEGREES + *
      + *
    • SQL semantic: DEGREES(expr)
    • + *
    • Since version: 3.4.0
    • + *
    + *
  • + *
  • Name: RADIANS + *
      + *
    • SQL semantic: RADIANS(expr)
    • + *
    • Since version: 3.4.0
    • + *
    + *
  • + *
  • Name: SIGN + *
      + *
    • SQL semantic: SIGN(expr)
    • + *
    • Since version: 3.4.0
    • + *
    + *
  • *
  • Name: WIDTH_BUCKET *
      *
    • SQL semantic: WIDTH_BUCKET(expr)
    • diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/ScanBuilder.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/ScanBuilder.java index 27ee534d804ff..f5ce604148b18 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/ScanBuilder.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/ScanBuilder.java @@ -23,7 +23,8 @@ * An interface for building the {@link Scan}. Implementations can mixin SupportsPushDownXYZ * interfaces to do operator push down, and keep the operator push down result in the returned * {@link Scan}. When pushing down operators, the push down order is: - * sample -> filter -> aggregate -> limit -> column pruning. + * sample -> filter -> aggregate -> limit/top-n(sort + limit) -> offset -> + * column pruning. * * @since 3.0.0 */ diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/SupportsPushDownLimit.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/SupportsPushDownLimit.java index 035154d08450a..8a725cd7ed7a8 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/SupportsPushDownLimit.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/SupportsPushDownLimit.java @@ -21,8 +21,8 @@ /** * A mix-in interface for {@link ScanBuilder}. Data sources can implement this interface to - * push down LIMIT. Please note that the combination of LIMIT with other operations - * such as AGGREGATE, GROUP BY, SORT BY, CLUSTER BY, DISTRIBUTE BY, etc. is NOT pushed down. + * push down LIMIT. We can push down LIMIT with many other operations if they follow the + * operator order we defined in {@link ScanBuilder}'s class doc. * * @since 3.3.0 */ diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/SupportsPushDownOffset.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/SupportsPushDownOffset.java new file mode 100644 index 0000000000000..ffa2cad371594 --- /dev/null +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/SupportsPushDownOffset.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.connector.read; + +import org.apache.spark.annotation.Evolving; + +/** + * A mix-in interface for {@link ScanBuilder}. Data sources can implement this interface to + * push down OFFSET. We can push down OFFSET with many other operations if they follow the + * operator order we defined in {@link ScanBuilder}'s class doc. + * + * @since 3.4.0 + */ +@Evolving +public interface SupportsPushDownOffset extends ScanBuilder { + + /** + * Pushes down OFFSET to the data source. + */ + boolean pushOffset(int offset); +} diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/SupportsPushDownTopN.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/SupportsPushDownTopN.java index cba1592c4fa14..83d15ba2296f5 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/SupportsPushDownTopN.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/SupportsPushDownTopN.java @@ -22,23 +22,22 @@ /** * A mix-in interface for {@link ScanBuilder}. Data sources can implement this interface to - * push down top N(query with ORDER BY ... LIMIT n). Please note that the combination of top N - * with other operations such as AGGREGATE, GROUP BY, CLUSTER BY, DISTRIBUTE BY, etc. - * is NOT pushed down. + * push down top N(query with ORDER BY ... LIMIT n). We can push down top N with many other + * operations if they follow the operator order we defined in {@link ScanBuilder}'s class doc. * * @since 3.3.0 */ @Evolving public interface SupportsPushDownTopN extends ScanBuilder { - /** - * Pushes down top N to the data source. - */ - boolean pushTopN(SortOrder[] orders, int limit); + /** + * Pushes down top N to the data source. + */ + boolean pushTopN(SortOrder[] orders, int limit); - /** - * Whether the top N is partially pushed or not. If it returns true, then Spark will do top N - * again. This method will only be called when {@link #pushTopN} returns true. - */ - default boolean isPartiallyPushed() { return true; } + /** + * Whether the top N is partially pushed or not. If it returns true, then Spark will do top N + * again. This method will only be called when {@link #pushTopN} returns true. + */ + default boolean isPartiallyPushed() { return true; } } diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/util/V2ExpressionSQLBuilder.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/util/V2ExpressionSQLBuilder.java index 396b1d9cdd034..43095807d95dd 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/util/V2ExpressionSQLBuilder.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/util/V2ExpressionSQLBuilder.java @@ -95,12 +95,37 @@ public String build(Expression expr) { return visitUnaryArithmetic(name, inputToSQL(e.children()[0])); case "ABS": case "COALESCE": + case "GREATEST": + case "LEAST": + case "RAND": + case "LOG": + case "LOG10": + case "LOG2": case "LN": case "EXP": case "POWER": case "SQRT": case "FLOOR": case "CEIL": + case "ROUND": + case "SIN": + case "SINH": + case "COS": + case "COSH": + case "TAN": + case "TANH": + case "COT": + case "ASIN": + case "ASINH": + case "ACOS": + case "ACOSH": + case "ATAN": + case "ATANH": + case "ATAN2": + case "CBRT": + case "DEGREES": + case "RADIANS": + case "SIGN": case "WIDTH_BUCKET": case "SUBSTRING": case "UPPER": diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index 14c44b0364f31..522eb07c096ee 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -393,20 +393,17 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog { val offset = offsetExpr.eval().asInstanceOf[Int] if (Int.MaxValue - limit < offset) { failAnalysis( - s"""The sum of limit and offset must not be greater than Int.MaxValue, - | but found limit = $limit, offset = $offset.""".stripMargin) + s""" + |The sum of the LIMIT clause and the OFFSET clause must not be greater than + |the maximum 32-bit integer value (2,147,483,647), + |but found limit = $limit, offset = $offset. + |""".stripMargin.replace("\n", " ")) } case _ => } case Offset(offsetExpr, _) => checkLimitLikeClause("offset", offsetExpr) - case o if !o.isInstanceOf[GlobalLimit] && !o.isInstanceOf[LocalLimit] - && o.children.exists(_.isInstanceOf[Offset]) => - failAnalysis( - s"""Only the OFFSET clause is allowed in the LIMIT clause, but the OFFSET - | clause found in: ${o.nodeName}.""".stripMargin) - case Tail(limitExpr, _) => checkLimitLikeClause("tail", limitExpr) case _: Union | _: SetOperation if operator.children.length > 1 => @@ -567,7 +564,6 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog { } } checkCollectedMetrics(plan) - checkOutermostOffset(plan) extendedCheckRules.foreach(_(plan)) plan.foreachUp { case o if !o.resolved => @@ -578,20 +574,6 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog { plan.setAnalyzed() } - /** - * Validate that the root node of query or subquery is [[Offset]]. - */ - private def checkOutermostOffset(plan: LogicalPlan): Unit = { - plan match { - case Offset(offsetExpr, _) => - checkLimitLikeClause("limit", offsetExpr) - failAnalysis( - s"""Only the OFFSET clause is allowed in the LIMIT clause, but the OFFSET - | clause is found to be the outermost node.""".stripMargin) - case _ => - } - } - /** * Validates subquery expressions in the plan. Upon failure, returns an user facing error. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index 1a57ee83fa3ef..0a53e9d73cd58 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -93,7 +93,7 @@ abstract class Optimizer(catalogManager: CatalogManager) OptimizeWindowFunctions, CollapseWindow, CombineFilters, - RewriteOffsets, + EliminateOffsets, EliminateLimits, CombineUnions, // Constant folding and strength reduction @@ -639,7 +639,7 @@ object RemoveNoopUnion extends Rule[LogicalPlan] { } /** - * Pushes down [[LocalLimit]] beneath UNION ALL and joins. + * Pushes down [[LocalLimit]] beneath UNION ALL, OFFSET and joins. */ object LimitPushDown extends Rule[LogicalPlan] { @@ -709,6 +709,14 @@ object LimitPushDown extends Rule[LogicalPlan] { // There is a Project between LocalLimit and Join if they do not have the same output. case LocalLimit(exp, project @ Project(_, join: Join)) => LocalLimit(exp, project.copy(child = pushLocalLimitThroughJoin(exp, join))) + // Push down limit 1 through Aggregate and turn Aggregate into Project if it is group only. + case Limit(le @ IntegerLiteral(1), a: Aggregate) if a.groupOnly => + Limit(le, Project(a.aggregateExpressions, LocalLimit(le, a.child))) + case Limit(le @ IntegerLiteral(1), p @ Project(_, a: Aggregate)) if a.groupOnly => + Limit(le, p.copy(child = Project(a.aggregateExpressions, LocalLimit(le, a.child)))) + // Merge offset value and limit value into LocalLimit and pushes down LocalLimit through Offset. + case LocalLimit(le, Offset(oe, grandChild)) => + Offset(oe, LocalLimit(Add(le, oe), grandChild)) } } @@ -1784,15 +1792,22 @@ object EliminateLimits extends Rule[LogicalPlan] { } /** - * Rewrite [[Offset]] as [[Limit]] or combines two adjacent [[Offset]] operators into one, - * merging the expressions into one single expression. + * This rule optimizes Offset operators by: + * 1. Eliminate [[Offset]] operators if offset == 0. + * 2. Replace [[Offset]] operators to empty [[LocalRelation]] + * if [[Offset]]'s child max row <= offset. + * 3. Combines two adjacent [[Offset]] operators into one, merging the + * expressions into one single expression. */ -object RewriteOffsets extends Rule[LogicalPlan] { +object EliminateOffsets extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { - case GlobalLimit(le, Offset(oe, grandChild)) => - GlobalLimitAndOffset(le, oe, grandChild) - case LocalLimit(le, Offset(oe, grandChild)) => - Offset(oe, LocalLimit(Add(le, oe), grandChild)) + case Offset(oe, child) if oe.foldable && oe.eval().asInstanceOf[Int] == 0 => + child + case Offset(oe, child) + if oe.foldable && child.maxRows.exists(_ <= oe.eval().asInstanceOf[Int]) => + LocalRelation(child.output, data = Seq.empty, isStreaming = child.isStreaming) + case Offset(oe1, Offset(oe2, child)) => + Offset(Add(oe1, oe2), child) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PropagateEmptyRelation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PropagateEmptyRelation.scala index 1a5a8c0fba45d..5b565793fac42 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PropagateEmptyRelation.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PropagateEmptyRelation.scala @@ -102,7 +102,7 @@ abstract class PropagateEmptyRelationBase extends Rule[LogicalPlan] with CastSup case _: Sort => empty(p) case _: GlobalLimit if !p.isStreaming => empty(p) case _: LocalLimit if !p.isStreaming => empty(p) - case _: Offset if !p.isStreaming => empty(p) + case _: Offset => empty(p) case _: Repartition => empty(p) case _: RepartitionByExpression => empty(p) // An aggregate with non-empty group expression will return one output row per group when the diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala index d3e6f285fa605..74f643ede4a9f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala @@ -915,6 +915,7 @@ object FoldablePropagation extends Rule[LogicalPlan] { case _: Sample => true case _: GlobalLimit => true case _: LocalLimit => true + case _: Offset => true case _: Generate => true case _: Distinct => true case _: AppendColumns => true diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala index 29610c3dcea4c..f16e724e5e4ce 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala @@ -1129,31 +1129,10 @@ case class Offset(offsetExpr: Expression, child: LogicalPlan) extends OrderPrese case _ => None } } - override protected def withNewChildInternal(newChild: LogicalPlan): Offset = copy(child = newChild) } -/** - * A global (coordinated) limit with offset. This operator can skip at most `offsetExpr` number and - * emit at most `limitExpr` number in total. - */ -case class GlobalLimitAndOffset( - limitExpr: Expression, - offsetExpr: Expression, - child: LogicalPlan) extends OrderPreservingUnaryNode { - override def output: Seq[Attribute] = child.output - override def maxRows: Option[Long] = { - limitExpr match { - case IntegerLiteral(limit) => Some(limit) - case _ => None - } - } - - override protected def withNewChildInternal(newChild: LogicalPlan): GlobalLimitAndOffset = - copy(child = newChild) -} - /** * A constructor for creating a pivot, which will later be converted to a [[Project]] * or an [[Aggregate]] during the query analysis. @@ -1265,6 +1244,30 @@ case class LocalLimit(limitExpr: Expression, child: LogicalPlan) extends OrderPr copy(child = newChild) } +object OffsetAndLimit { + def unapply(p: GlobalLimit): Option[(Int, Int, LogicalPlan)] = { + p match { + // Optimizer pushes local limit through offset, so we need to match the plan this way. + case GlobalLimit(IntegerLiteral(globalLimit), + Offset(IntegerLiteral(offset), + LocalLimit(IntegerLiteral(localLimit), child))) + if globalLimit + offset == localLimit => + Some((offset, globalLimit, child)) + case _ => None + } + } +} + +object LimitAndOffset { + def unapply(p: Offset): Option[(Int, Int, LogicalPlan)] = { + p match { + case Offset(IntegerLiteral(offset), Limit(IntegerLiteral(limit), child)) => + Some((limit, offset, child)) + case _ => None + } + } +} + /** * This is similar with [[Limit]] except: * diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/SizeInBytesOnlyStatsPlanVisitor.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/SizeInBytesOnlyStatsPlanVisitor.scala index 3d115fe77176a..d97bb5be36fb1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/SizeInBytesOnlyStatsPlanVisitor.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/SizeInBytesOnlyStatsPlanVisitor.scala @@ -93,7 +93,7 @@ object SizeInBytesOnlyStatsPlanVisitor extends LogicalPlanVisitor[Statistics] { override def visitOffset(p: Offset): Statistics = { val offset = p.offsetExpr.eval().asInstanceOf[Int] val childStats = p.child.stats - val rowCount: BigInt = childStats.rowCount.map(_.-(offset).max(0)).getOrElse(0) + val rowCount: BigInt = childStats.rowCount.map(c => c - offset).map(_.max(0)).getOrElse(0) Statistics( sizeInBytes = EstimationUtils.getOutputSize(p.output, rowCount, childStats.attributeStats), rowCount = Some(rowCount)) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala index 54cdd91c93ea5..9d546b69d0fe6 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala @@ -30,6 +30,7 @@ import org.apache.spark.sql.catalyst.plans.{Cross, LeftOuter, RightOuter} import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, GenericArrayData, MapData} import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.types.UTF8String private[sql] case class GroupableData(data: Int) { def getData: Int = data @@ -549,6 +550,38 @@ class AnalysisErrorSuite extends AnalysisTest { "The limit expression must be equal to or greater than 0, but got -1" :: Nil ) + errorTest( + "an evaluated offset class must not be string", + testRelation.offset(Literal(UTF8String.fromString("abc"), StringType)), + "The offset expression must be integer type, but got string" :: Nil + ) + + errorTest( + "an evaluated offset class must not be long", + testRelation.offset(Literal(10L, LongType)), + "The offset expression must be integer type, but got bigint" :: Nil + ) + + errorTest( + "an evaluated offset class must not be null", + testRelation.offset(Literal(null, IntegerType)), + "The evaluated offset expression must not be null, but got " :: Nil + ) + + errorTest( + "num_rows in offset clause must be equal to or greater than 0", + testRelation.offset(-1), + "The offset expression must be equal to or greater than 0, but got -1" :: Nil + ) + + errorTest( + "the sum of num_rows in limit clause and num_rows in offset clause less than Int.MaxValue", + testRelation.offset(Literal(2000000000, IntegerType)).limit(Literal(1000000000, IntegerType)), + "The sum of the LIMIT clause and the OFFSET clause must not be greater than" + + " the maximum 32-bit integer value (2,147,483,647)," + + " but found limit = 1000000000, offset = 2000000000." :: Nil + ) + errorTest( "more than one generators in SELECT", listRelation.select(Explode($"list"), Explode($"list")), diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/EliminateOffsetsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/EliminateOffsetsSuite.scala new file mode 100644 index 0000000000000..d8c0199ac37dc --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/EliminateOffsetsSuite.scala @@ -0,0 +1,100 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.optimizer + +import org.apache.spark.sql.Row +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.dsl.plans._ +import org.apache.spark.sql.catalyst.expressions.{Add, Literal} +import org.apache.spark.sql.catalyst.plans.PlanTest +import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan} +import org.apache.spark.sql.catalyst.rules.RuleExecutor + +class EliminateOffsetsSuite extends PlanTest { + object Optimize extends RuleExecutor[LogicalPlan] { + val batches = + Batch("Eliminate Offset", FixedPoint(10), EliminateOffsets) :: Nil + } + + val testRelation = LocalRelation.fromExternalRows( + Seq("a".attr.int, "b".attr.int, "c".attr.int), + 1.to(10).map(_ => Row(1, 2, 3)) + ) + + test("Offsets: eliminate Offset operators if offset == 0") { + val originalQuery = + testRelation + .select($"a") + .offset(0) + + val optimized = Optimize.execute(originalQuery.analyze) + val correctAnswer = + testRelation + .select($"a") + .analyze + + comparePlans(optimized, correctAnswer) + } + + test("Offsets: cannot eliminate Offset operators if offset > 0") { + val originalQuery = + testRelation + .select($"a") + .offset(2) + + val optimized = Optimize.execute(originalQuery.analyze) + val correctAnswer = + testRelation + .select($"a") + .offset(2) + .analyze + + comparePlans(optimized, correctAnswer) + } + + test("Replace Offset operators to empty LocalRelation if child max row <= offset") { + val child = testRelation.select($"a").analyze + val originalQuery = child.offset(10) + + val optimized = Optimize.execute(originalQuery.analyze) + val correctAnswer = + LocalRelation(child.output, data = Seq.empty, isStreaming = child.isStreaming).analyze + + comparePlans(optimized, correctAnswer) + } + + test("Cannot replace Offset operators to empty LocalRelation if child max row > offset") { + val child = testRelation.select($"a").analyze + val originalQuery = child.offset(3) + + val optimized = Optimize.execute(originalQuery.analyze) + val correctAnswer = originalQuery.analyze + + comparePlans(optimized, correctAnswer) + } + + test("Combines Offset operators") { + val child = testRelation.select($"a").analyze + val originalQuery = child.offset(2).offset(3) + + val optimized = Optimize.execute(originalQuery.analyze) + val correctAnswer = child.offset(Add(Literal(3), Literal(2))).analyze + + comparePlans(optimized, correctAnswer) + } +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LimitPushdownSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LimitPushdownSuite.scala index c2503e362c8c8..472871a813c52 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LimitPushdownSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LimitPushdownSuite.scala @@ -239,4 +239,10 @@ class LimitPushdownSuite extends PlanTest { Limit(5, LocalLimit(5, x).join(y, LeftOuter, joinCondition).select("x.a".attr)).analyze comparePlans(optimized, correctAnswer) } + + test("Push down limit 1 through Offset") { + comparePlans( + Optimize.execute(testRelation.offset(2).limit(1).analyze), + GlobalLimit(1, Offset(2, LocalLimit(3, testRelation))).analyze) + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/BasicStatsEstimationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/BasicStatsEstimationSuite.scala index 7fa9a3141d06a..1097dafcafedc 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/BasicStatsEstimationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/BasicStatsEstimationSuite.scala @@ -185,6 +185,22 @@ class BasicStatsEstimationSuite extends PlanTest with StatsEstimationTestBase { expectedStatsCboOff = windowsStats) } + test("offset estimation: offset < child's rowCount") { + val offset = Offset(Literal(2), plan) + checkStats(offset, Statistics(sizeInBytes = 96, rowCount = Some(8))) + } + + test("offset estimation: offset > child's rowCount") { + val offset = Offset(Literal(20), plan) + checkStats(offset, Statistics(sizeInBytes = 1, rowCount = Some(0))) + } + + test("offset estimation: offset = 0") { + val offset = Offset(Literal(0), plan) + // Offset is equal to zero, so Offset's stats is equal to its child's stats. + checkStats(offset, plan.stats.copy(attributeStats = AttributeMap(Nil))) + } + test("limit estimation: limit < child's rowCount") { val localLimit = LocalLimit(Literal(2), plan) val globalLimit = GlobalLimit(Literal(2), plan) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index 785566f1ff7f4..96204aa2f071b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -1952,9 +1952,16 @@ class Dataset[T] private[sql]( Limit(Literal(n), logicalPlan) } - def limitRange(start: Int, end: Int): Dataset[T] = withTypedPlan { - Limit(Literal(end - start), Offset(Literal(start), logicalPlan)) + /** + * Returns a new Dataset by skipping the first `n` rows. + * + * @group typedrel + * @since 3.4.0 + */ + def offset(n: Int): Dataset[T] = withTypedPlan { + Offset(Literal(n), logicalPlan) } + /** * Returns a new Dataset containing union of rows in this Dataset and another Dataset. * diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/util/V2ExpressionBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/util/V2ExpressionBuilder.scala index 120b204413561..1a9b9202cbe97 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/util/V2ExpressionBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/util/V2ExpressionBuilder.scala @@ -17,19 +17,15 @@ package org.apache.spark.sql.catalyst.util -import org.apache.spark.sql.catalyst.expressions.{Abs, Add, And, BinaryComparison, BinaryOperator, BitwiseAnd, BitwiseNot, BitwiseOr, BitwiseXor, CaseWhen, Cast, Ceil, Coalesce, Contains, Divide, EndsWith, EqualTo, Exp, Expression, Floor, In, InSet, IsNotNull, IsNull, Literal, Log, Lower, Multiply, Not, Or, Overlay, Pow, Predicate, Remainder, Sqrt, StartsWith, StringPredicate, StringTranslate, StringTrim, StringTrimLeft, StringTrimRight, Substring, Subtract, UnaryMinus, Upper, WidthBucket} +import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.connector.expressions.{Cast => V2Cast, Expression => V2Expression, FieldReference, GeneralScalarExpression, LiteralValue} import org.apache.spark.sql.connector.expressions.filter.{AlwaysFalse, AlwaysTrue, And => V2And, Not => V2Not, Or => V2Or, Predicate => V2Predicate} -import org.apache.spark.sql.execution.datasources.PushableColumn import org.apache.spark.sql.types.BooleanType /** * The builder to generate V2 expressions from catalyst expressions. */ -class V2ExpressionBuilder( - e: Expression, nestedPredicatePushdownEnabled: Boolean = false, isPredicate: Boolean = false) { - - val pushableColumn = PushableColumn(nestedPredicatePushdownEnabled) +class V2ExpressionBuilder(e: Expression, isPredicate: Boolean = false) { def build(): Option[V2Expression] = generateExpression(e, isPredicate) @@ -49,12 +45,8 @@ class V2ExpressionBuilder( case Literal(true, BooleanType) => Some(new AlwaysTrue()) case Literal(false, BooleanType) => Some(new AlwaysFalse()) case Literal(value, dataType) => Some(LiteralValue(value, dataType)) - case col @ pushableColumn(name) => - val ref = if (nestedPredicatePushdownEnabled) { - FieldReference(name) - } else { - FieldReference.column(name) - } + case col @ ColumnOrField(nameParts) => + val ref = FieldReference(nameParts) if (isPredicate && col.dataType.isInstanceOf[BooleanType]) { Some(new V2Predicate("=", Array(ref, LiteralValue(true, BooleanType)))) } else { @@ -107,6 +99,39 @@ class V2ExpressionBuilder( } else { None } + case Greatest(children) => + val childrenExpressions = children.flatMap(generateExpression(_)) + if (children.length == childrenExpressions.length) { + Some(new GeneralScalarExpression("GREATEST", childrenExpressions.toArray[V2Expression])) + } else { + None + } + case Least(children) => + val childrenExpressions = children.flatMap(generateExpression(_)) + if (children.length == childrenExpressions.length) { + Some(new GeneralScalarExpression("LEAST", childrenExpressions.toArray[V2Expression])) + } else { + None + } + case Rand(child, hideSeed) => + if (hideSeed) { + Some(new GeneralScalarExpression("RAND", Array.empty[V2Expression])) + } else { + generateExpression(child) + .map(v => new GeneralScalarExpression("RAND", Array[V2Expression](v))) + } + case log: Logarithm => + val l = generateExpression(log.left) + val r = generateExpression(log.right) + if (l.isDefined && r.isDefined) { + Some(new GeneralScalarExpression("LOG", Array[V2Expression](l.get, r.get))) + } else { + None + } + case Log10(child) => generateExpression(child) + .map(v => new GeneralScalarExpression("LOG10", Array[V2Expression](v))) + case Log2(child) => generateExpression(child) + .map(v => new GeneralScalarExpression("LOG2", Array[V2Expression](v))) case Log(child) => generateExpression(child) .map(v => new GeneralScalarExpression("LN", Array[V2Expression](v))) case Exp(child) => generateExpression(child) @@ -125,6 +150,56 @@ class V2ExpressionBuilder( .map(v => new GeneralScalarExpression("FLOOR", Array[V2Expression](v))) case Ceil(child) => generateExpression(child) .map(v => new GeneralScalarExpression("CEIL", Array[V2Expression](v))) + case round: Round => + val l = generateExpression(round.left) + val r = generateExpression(round.right) + if (l.isDefined && r.isDefined) { + Some(new GeneralScalarExpression("ROUND", Array[V2Expression](l.get, r.get))) + } else { + None + } + case Sin(child) => generateExpression(child) + .map(v => new GeneralScalarExpression("SIN", Array[V2Expression](v))) + case Sinh(child) => generateExpression(child) + .map(v => new GeneralScalarExpression("SINH", Array[V2Expression](v))) + case Cos(child) => generateExpression(child) + .map(v => new GeneralScalarExpression("COS", Array[V2Expression](v))) + case Cosh(child) => generateExpression(child) + .map(v => new GeneralScalarExpression("COSH", Array[V2Expression](v))) + case Tan(child) => generateExpression(child) + .map(v => new GeneralScalarExpression("TAN", Array[V2Expression](v))) + case Tanh(child) => generateExpression(child) + .map(v => new GeneralScalarExpression("TANH", Array[V2Expression](v))) + case Cot(child) => generateExpression(child) + .map(v => new GeneralScalarExpression("COT", Array[V2Expression](v))) + case Asin(child) => generateExpression(child) + .map(v => new GeneralScalarExpression("ASIN", Array[V2Expression](v))) + case Asinh(child) => generateExpression(child) + .map(v => new GeneralScalarExpression("ASINH", Array[V2Expression](v))) + case Acos(child) => generateExpression(child) + .map(v => new GeneralScalarExpression("ACOS", Array[V2Expression](v))) + case Acosh(child) => generateExpression(child) + .map(v => new GeneralScalarExpression("ACOSH", Array[V2Expression](v))) + case Atan(child) => generateExpression(child) + .map(v => new GeneralScalarExpression("ATAN", Array[V2Expression](v))) + case Atanh(child) => generateExpression(child) + .map(v => new GeneralScalarExpression("ATANH", Array[V2Expression](v))) + case atan2: Atan2 => + val l = generateExpression(atan2.left) + val r = generateExpression(atan2.right) + if (l.isDefined && r.isDefined) { + Some(new GeneralScalarExpression("ATAN2", Array[V2Expression](l.get, r.get))) + } else { + None + } + case Cbrt(child) => generateExpression(child) + .map(v => new GeneralScalarExpression("CBRT", Array[V2Expression](v))) + case ToDegrees(child) => generateExpression(child) + .map(v => new GeneralScalarExpression("DEGREES", Array[V2Expression](v))) + case ToRadians(child) => generateExpression(child) + .map(v => new GeneralScalarExpression("RADIANS", Array[V2Expression](v))) + case Signum(child) => generateExpression(child) + .map(v => new GeneralScalarExpression("SIGN", Array[V2Expression](v))) case wb: WidthBucket => val childrenExpressions = wb.children.flatMap(generateExpression(_)) if (childrenExpressions.length == wb.children.length) { @@ -203,6 +278,13 @@ class V2ExpressionBuilder( } else { None } + case iff: If => + val childrenExpressions = iff.children.flatMap(generateExpression(_)) + if (iff.children.length == childrenExpressions.length) { + Some(new GeneralScalarExpression("CASE_WHEN", childrenExpressions.toArray[V2Expression])) + } else { + None + } case substring: Substring => val children = if (substring.len == Literal(Integer.MAX_VALUE)) { Seq(substring.str, substring.pos) @@ -266,3 +348,12 @@ class V2ExpressionBuilder( case _ => None } } + +object ColumnOrField { + def unapply(e: Expression): Option[Seq[String]] = e match { + case a: Attribute => Some(Seq(a.name)) + case s: GetStructField => + unapply(s.child).map(_ :+ s.childSchema(s.ordinal).name) + case _ => None + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala index 86edb5acbe292..1181ad05bb413 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala @@ -148,9 +148,11 @@ case class RowDataSourceScanExec( s"ORDER BY ${seqToString(pushedDownOperators.sortValues.map(_.describe()))}" + s" LIMIT ${pushedDownOperators.limit.get}" Some("PushedTopN" -> pushedTopN) - } else { - pushedDownOperators.limit.map(value => "PushedLimit" -> s"LIMIT $value") - } + } else { + pushedDownOperators.limit.map(value => "PushedLimit" -> s"LIMIT $value") + } + + val offsetInfo = pushedDownOperators.offset.map(value => "PushedOffset" -> s"OFFSET $value") val pushedFilters = if (pushedDownOperators.pushedPredicates.nonEmpty) { seqToString(pushedDownOperators.pushedPredicates.map(_.describe())) @@ -164,6 +166,7 @@ case class RowDataSourceScanExec( Map("PushedAggregates" -> seqToString(v.aggregateExpressions.map(_.describe())), "PushedGroupByExpressions" -> seqToString(v.groupByExpressions.map(_.describe())))} ++ topNOrLimitInfo ++ + offsetInfo ++ pushedDownOperators.sample.map(v => "PushedSample" -> s"SAMPLE (${(v.upperBound - v.lowerBound) * 100}) ${v.withReplacement} SEED(${v.seed})" ) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index 8b5ee84fb6e98..422f1f041a58b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -80,40 +80,56 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { */ object SpecialLimits extends Strategy { override def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { - case ReturnAnswer(rootPlan) => rootPlan match { - case Limit(IntegerLiteral(limit), Sort(order, true, child)) - if limit < conf.topKSortFallbackThreshold => - TakeOrderedAndProjectExec(limit, 0, order, child.output, planLater(child)) :: Nil - case Limit(IntegerLiteral(limit), Project(projectList, Sort(order, true, child))) - if limit < conf.topKSortFallbackThreshold => - TakeOrderedAndProjectExec(limit, 0, order, projectList, planLater(child)) :: Nil + // Call `planTakeOrdered` first which matches a larger plan. + case ReturnAnswer(rootPlan) => planTakeOrdered(rootPlan).getOrElse(rootPlan match { + // We should match the combination of limit and offset first, to get the optimal physical + // plan, instead of planning limit and offset separately. + case LimitAndOffset(limit, offset, child) => + CollectLimitExec(limit = limit, child = planLater(child), offset = offset) + case OffsetAndLimit(offset, limit, child) => + // 'Offset a' then 'Limit b' is the same as 'Limit a + b' then 'Offset a'. + CollectLimitExec(limit = offset + limit, child = planLater(child), offset = offset) case Limit(IntegerLiteral(limit), child) => - CollectLimitExec(limit, 0, planLater(child)) :: Nil - case GlobalLimitAndOffset( - IntegerLiteral(limit), - IntegerLiteral(offset), - Sort(order, true, child)) + CollectLimitExec(limit = limit, child = planLater(child)) + case logical.Offset(IntegerLiteral(offset), child) => + CollectLimitExec(child = planLater(child), offset = offset) + case Tail(IntegerLiteral(limit), child) => + CollectTailExec(limit, planLater(child)) + case other => planLater(other) + }) :: Nil + + case other => planTakeOrdered(other).toSeq + } + + private def planTakeOrdered(plan: LogicalPlan): Option[SparkPlan] = plan match { + // We should match the combination of limit and offset first, to get the optimal physical + // plan, instead of planning limit and offset separately. + case LimitAndOffset(limit, offset, Sort(order, true, child)) if limit < conf.topKSortFallbackThreshold => - TakeOrderedAndProjectExec(limit, offset, order, child.output, planLater(child)) :: Nil - case GlobalLimitAndOffset( - IntegerLiteral(limit), - IntegerLiteral(offset), - Project(projectList, Sort(order, true, child))) + Some(TakeOrderedAndProjectExec( + limit, order, child.output, planLater(child), offset)) + case LimitAndOffset(limit, offset, Project(projectList, Sort(order, true, child))) if limit < conf.topKSortFallbackThreshold => - TakeOrderedAndProjectExec(limit, offset, order, projectList, planLater(child)) :: Nil - case GlobalLimitAndOffset(IntegerLiteral(limit), IntegerLiteral(offset), child) => - CollectLimitExec(limit, offset, planLater(child)) :: Nil - case Tail(IntegerLiteral(limit), child) => - CollectTailExec(limit, planLater(child)) :: Nil - case other => planLater(other) :: Nil - } + Some(TakeOrderedAndProjectExec( + limit, order, projectList, planLater(child), offset)) + // 'Offset a' then 'Limit b' is the same as 'Limit a + b' then 'Offset a'. + case OffsetAndLimit(offset, limit, Sort(order, true, child)) + if offset + limit < conf.topKSortFallbackThreshold => + Some(TakeOrderedAndProjectExec( + offset + limit, order, child.output, planLater(child), offset)) + case OffsetAndLimit(offset, limit, Project(projectList, Sort(order, true, child))) + if offset + limit < conf.topKSortFallbackThreshold => + Some(TakeOrderedAndProjectExec( + offset + limit, order, projectList, planLater(child), offset)) case Limit(IntegerLiteral(limit), Sort(order, true, child)) if limit < conf.topKSortFallbackThreshold => - TakeOrderedAndProjectExec(limit, 0, order, child.output, planLater(child)) :: Nil + Some(TakeOrderedAndProjectExec( + limit, order, child.output, planLater(child))) case Limit(IntegerLiteral(limit), Project(projectList, Sort(order, true, child))) if limit < conf.topKSortFallbackThreshold => - TakeOrderedAndProjectExec(limit, 0, order, projectList, planLater(child)) :: Nil - case _ => Nil + Some(TakeOrderedAndProjectExec( + limit, order, projectList, planLater(child))) + case _ => None } } @@ -771,12 +787,19 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { case logical.LocalRelation(output, data, _) => LocalTableScanExec(output, data) :: Nil case CommandResult(output, _, plan, data) => CommandResultExec(output, plan, data) :: Nil + // We should match the combination of limit and offset first, to get the optimal physical + // plan, instead of planning limit and offset separately. + case LimitAndOffset(limit, offset, child) => + GlobalLimitExec(limit, planLater(child), offset) :: Nil + case OffsetAndLimit(offset, limit, child) => + // 'Offset a' then 'Limit b' is the same as 'Limit a + b' then 'Offset a'. + GlobalLimitExec(limit = offset + limit, child = planLater(child), offset = offset) :: Nil case logical.LocalLimit(IntegerLiteral(limit), child) => execution.LocalLimitExec(limit, planLater(child)) :: Nil case logical.GlobalLimit(IntegerLiteral(limit), child) => execution.GlobalLimitExec(limit, planLater(child)) :: Nil - case logical.GlobalLimitAndOffset(IntegerLiteral(limit), IntegerLiteral(offset), child) => - execution.GlobalLimitAndOffsetExec(limit, offset, planLater(child)) :: Nil + case logical.Offset(IntegerLiteral(offset), child) => + GlobalLimitExec(child = planLater(child), offset = offset) :: Nil case union: logical.Union => execution.UnionExec(union.children.map(planLater)) :: Nil case g @ logical.Generate(generator, _, outer, _, _, child) => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala index 7209a61d39a31..8d8e2c26e279e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala @@ -337,7 +337,7 @@ object DataSourceStrategy l.output.toStructType, Set.empty, Set.empty, - PushedDownOperators(None, None, None, Seq.empty, Seq.empty), + PushedDownOperators(None, None, None, None, Seq.empty, Seq.empty), toCatalystRDD(l, baseRelation.buildScan()), baseRelation, None) :: Nil @@ -411,7 +411,7 @@ object DataSourceStrategy requestedColumns.toStructType, pushedFilters.toSet, handledFilters, - PushedDownOperators(None, None, None, Seq.empty, Seq.empty), + PushedDownOperators(None, None, None, None, Seq.empty, Seq.empty), scanBuilder(requestedColumns, candidatePredicates, pushedFilters), relation.relation, relation.catalogTable.map(_.identifier)) @@ -434,7 +434,7 @@ object DataSourceStrategy requestedColumns.toStructType, pushedFilters.toSet, handledFilters, - PushedDownOperators(None, None, None, Seq.empty, Seq.empty), + PushedDownOperators(None, None, None, None, Seq.empty, Seq.empty), scanBuilder(requestedColumns, candidatePredicates, pushedFilters), relation.relation, relation.catalogTable.map(_.identifier)) @@ -770,8 +770,8 @@ object DataSourceStrategy } protected[sql] def translateSortOrders(sortOrders: Seq[SortOrder]): Seq[V2SortOrder] = { - def translateOortOrder(sortOrder: SortOrder): Option[V2SortOrder] = sortOrder match { - case SortOrder(PushableColumnWithoutNestedColumn(name), directionV1, nullOrderingV1, _) => + def translateSortOrder(sortOrder: SortOrder): Option[V2SortOrder] = sortOrder match { + case SortOrder(PushableExpression(expr), directionV1, nullOrderingV1, _) => val directionV2 = directionV1 match { case Ascending => SortDirection.ASCENDING case Descending => SortDirection.DESCENDING @@ -780,11 +780,11 @@ object DataSourceStrategy case NullsFirst => NullOrdering.NULLS_FIRST case NullsLast => NullOrdering.NULLS_LAST } - Some(SortValue(FieldReference(name), directionV2, nullOrderingV2)) + Some(SortValue(expr, directionV2, nullOrderingV2)) case _ => None } - sortOrders.flatMap(translateOortOrder) + sortOrders.flatMap(translateSortOrder) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCOptions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCOptions.scala index 8e047d7f7c7d5..5cf38eb803216 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCOptions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCOptions.scala @@ -195,6 +195,10 @@ class JDBCOptions( // This only applies to Data Source V2 JDBC val pushDownLimit = parameters.getOrElse(JDBC_PUSHDOWN_LIMIT, "false").toBoolean + // An option to allow/disallow pushing down OFFSET into V2 JDBC data source + // This only applies to Data Source V2 JDBC + val pushDownOffset = parameters.getOrElse(JDBC_PUSHDOWN_OFFSET, "false").toBoolean + // An option to allow/disallow pushing down TABLESAMPLE into JDBC data source // This only applies to Data Source V2 JDBC val pushDownTableSample = parameters.getOrElse(JDBC_PUSHDOWN_TABLESAMPLE, "false").toBoolean @@ -272,6 +276,7 @@ object JDBCOptions { val JDBC_PUSHDOWN_PREDICATE = newOption("pushDownPredicate") val JDBC_PUSHDOWN_AGGREGATE = newOption("pushDownAggregate") val JDBC_PUSHDOWN_LIMIT = newOption("pushDownLimit") + val JDBC_PUSHDOWN_OFFSET = newOption("pushDownOffset") val JDBC_PUSHDOWN_TABLESAMPLE = newOption("pushDownTableSample") val JDBC_KEYTAB = newOption("keytab") val JDBC_PRINCIPAL = newOption("principal") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRDD.scala index b30b460ac67db..8aba40e7548af 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRDD.scala @@ -25,7 +25,6 @@ import org.apache.spark.{InterruptibleIterator, Partition, SparkContext, TaskCon import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.connector.expressions.SortOrder import org.apache.spark.sql.connector.expressions.filter.Predicate import org.apache.spark.sql.execution.datasources.v2.TableSampleInfo import org.apache.spark.sql.jdbc.{JdbcDialect, JdbcDialects} @@ -123,7 +122,8 @@ object JDBCRDD extends Logging { groupByColumns: Option[Array[String]] = None, sample: Option[TableSampleInfo] = None, limit: Int = 0, - sortOrders: Array[SortOrder] = Array.empty[SortOrder]): RDD[InternalRow] = { + sortOrders: Array[String] = Array.empty[String], + offset: Int = 0): RDD[InternalRow] = { val url = options.url val dialect = JdbcDialects.get(url) val quotedColumns = if (groupByColumns.isEmpty) { @@ -144,7 +144,8 @@ object JDBCRDD extends Logging { groupByColumns, sample, limit, - sortOrders) + sortOrders, + offset) } // scalastyle:on argcount } @@ -166,7 +167,8 @@ private[jdbc] class JDBCRDD( groupByColumns: Option[Array[String]], sample: Option[TableSampleInfo], limit: Int, - sortOrders: Array[SortOrder]) + sortOrders: Array[String], + offset: Int) extends RDD[InternalRow](sc, Nil) { /** @@ -216,7 +218,7 @@ private[jdbc] class JDBCRDD( private def getOrderByClause: String = { if (sortOrders.nonEmpty) { - s" ORDER BY ${sortOrders.map(_.describe()).mkString(", ")}" + s" ORDER BY ${sortOrders.mkString(", ")}" } else { "" } @@ -304,9 +306,10 @@ private[jdbc] class JDBCRDD( } val myLimitClause: String = dialect.getLimitClause(limit) + val myOffsetClause: String = dialect.getOffsetClause(offset) val sqlText = s"SELECT $columnList FROM ${options.tableOrQuery} $myTableSampleClause" + - s" $myWhereClause $getGroupByClause $getOrderByClause $myLimitClause" + s" $myWhereClause $getGroupByClause $getOrderByClause $myLimitClause $myOffsetClause" stmt = conn.prepareStatement(sqlText, ResultSet.TYPE_FORWARD_ONLY, ResultSet.CONCUR_READ_ONLY) stmt.setFetchSize(options.fetchSize) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala index 0f1a1b6dc667b..fc439ac5ea1f3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala @@ -27,7 +27,6 @@ import org.apache.spark.sql.{DataFrame, Row, SaveMode, SparkSession, SQLContext} import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.util.{DateFormatter, DateTimeUtils, TimestampFormatter} import org.apache.spark.sql.catalyst.util.DateTimeUtils.{getZoneId, stringToDate, stringToTimestamp} -import org.apache.spark.sql.connector.expressions.SortOrder import org.apache.spark.sql.connector.expressions.filter.Predicate import org.apache.spark.sql.errors.QueryCompilationErrors import org.apache.spark.sql.execution.datasources.v2.TableSampleInfo @@ -305,7 +304,8 @@ private[sql] case class JDBCRelation( groupByColumns: Option[Array[String]], tableSample: Option[TableSampleInfo], limit: Int, - sortOrders: Array[SortOrder]): RDD[Row] = { + sortOrders: Array[String], + offset: Int): RDD[Row] = { // Rely on a type erasure hack to pass RDD[InternalRow] back as RDD[Row] JDBCRDD.scanTable( sparkSession.sparkContext, @@ -318,7 +318,8 @@ private[sql] case class JDBCRelation( groupByColumns, tableSample, limit, - sortOrders).asInstanceOf[RDD[Row]] + sortOrders, + offset).asInstanceOf[RDD[Row]] } override def insert(data: DataFrame, overwrite: Boolean): Unit = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala index f267a03cbe218..7430cb43db985 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala @@ -434,12 +434,9 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat private[sql] object DataSourceV2Strategy { - private def translateLeafNodeFilterV2( - predicate: Expression, - supportNestedPredicatePushdown: Boolean): Option[Predicate] = { - val pushablePredicate = PushablePredicate(supportNestedPredicatePushdown) + private def translateLeafNodeFilterV2(predicate: Expression): Option[Predicate] = { predicate match { - case pushablePredicate(expr) => Some(expr) + case PushablePredicate(expr) => Some(expr) case _ => None } } @@ -449,10 +446,8 @@ private[sql] object DataSourceV2Strategy { * * @return a `Some[Filter]` if the input [[Expression]] is convertible, otherwise a `None`. */ - protected[sql] def translateFilterV2( - predicate: Expression, - supportNestedPredicatePushdown: Boolean): Option[Predicate] = { - translateFilterV2WithMapping(predicate, None, supportNestedPredicatePushdown) + protected[sql] def translateFilterV2(predicate: Expression): Option[Predicate] = { + translateFilterV2WithMapping(predicate, None) } /** @@ -466,8 +461,7 @@ private[sql] object DataSourceV2Strategy { */ protected[sql] def translateFilterV2WithMapping( predicate: Expression, - translatedFilterToExpr: Option[mutable.HashMap[Predicate, Expression]], - nestedPredicatePushdownEnabled: Boolean) + translatedFilterToExpr: Option[mutable.HashMap[Predicate, Expression]]) : Option[Predicate] = { predicate match { case And(left, right) => @@ -481,26 +475,21 @@ private[sql] object DataSourceV2Strategy { // Pushing one leg of AND down is only safe to do at the top level. // You can see ParquetFilters' createFilter for more details. for { - leftFilter <- translateFilterV2WithMapping( - left, translatedFilterToExpr, nestedPredicatePushdownEnabled) - rightFilter <- translateFilterV2WithMapping( - right, translatedFilterToExpr, nestedPredicatePushdownEnabled) + leftFilter <- translateFilterV2WithMapping(left, translatedFilterToExpr) + rightFilter <- translateFilterV2WithMapping(right, translatedFilterToExpr) } yield new V2And(leftFilter, rightFilter) case Or(left, right) => for { - leftFilter <- translateFilterV2WithMapping( - left, translatedFilterToExpr, nestedPredicatePushdownEnabled) - rightFilter <- translateFilterV2WithMapping( - right, translatedFilterToExpr, nestedPredicatePushdownEnabled) + leftFilter <- translateFilterV2WithMapping(left, translatedFilterToExpr) + rightFilter <- translateFilterV2WithMapping(right, translatedFilterToExpr) } yield new V2Or(leftFilter, rightFilter) case Not(child) => - translateFilterV2WithMapping(child, translatedFilterToExpr, nestedPredicatePushdownEnabled) - .map(new V2Not(_)) + translateFilterV2WithMapping(child, translatedFilterToExpr).map(new V2Not(_)) case other => - val filter = translateLeafNodeFilterV2(other, nestedPredicatePushdownEnabled) + val filter = translateLeafNodeFilterV2(other) if (filter.isDefined && translatedFilterToExpr.isDefined) { translatedFilterToExpr.get(filter.get) = predicate } @@ -532,10 +521,9 @@ private[sql] object DataSourceV2Strategy { /** * Get the expression of DS V2 to represent catalyst predicate that can be pushed down. */ -case class PushablePredicate(nestedPredicatePushdownEnabled: Boolean) { - +object PushablePredicate { def unapply(e: Expression): Option[Predicate] = - new V2ExpressionBuilder(e, nestedPredicatePushdownEnabled, true).build().map { v => + new V2ExpressionBuilder(e, true).build().map { v => assert(v.isInstanceOf[Predicate]) v.asInstanceOf[Predicate] } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PushDownUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PushDownUtils.scala index 34bc384b318d9..66d26446f40a6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PushDownUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PushDownUtils.scala @@ -23,7 +23,7 @@ import org.apache.spark.sql.catalyst.expressions.{AttributeReference, AttributeS import org.apache.spark.sql.catalyst.util.CharVarcharUtils import org.apache.spark.sql.connector.expressions.SortOrder import org.apache.spark.sql.connector.expressions.filter.Predicate -import org.apache.spark.sql.connector.read.{Scan, ScanBuilder, SupportsPushDownFilters, SupportsPushDownLimit, SupportsPushDownRequiredColumns, SupportsPushDownTableSample, SupportsPushDownTopN, SupportsPushDownV2Filters} +import org.apache.spark.sql.connector.read.{Scan, ScanBuilder, SupportsPushDownFilters, SupportsPushDownLimit, SupportsPushDownOffset, SupportsPushDownRequiredColumns, SupportsPushDownTableSample, SupportsPushDownTopN, SupportsPushDownV2Filters} import org.apache.spark.sql.execution.datasources.DataSourceStrategy import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources @@ -80,7 +80,7 @@ object PushDownUtils extends PredicateHelper { for (filterExpr <- filters) { val translated = DataSourceV2Strategy.translateFilterV2WithMapping( - filterExpr, Some(translatedFilterToExpr), nestedPredicatePushdownEnabled = true) + filterExpr, Some(translatedFilterToExpr)) if (translated.isEmpty) { untranslatableExprs += filterExpr } else { @@ -131,6 +131,19 @@ object PushDownUtils extends PredicateHelper { } } + /** + * Pushes down OFFSET to the data source Scan. + * + * @return the Boolean value represents whether to push down. + */ + def pushOffset(scanBuilder: ScanBuilder, offset: Int): Boolean = { + scanBuilder match { + case s: SupportsPushDownOffset => + s.pushOffset(offset) + case _ => false + } + } + /** * Pushes down top N to the data source Scan. * diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PushedDownOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PushedDownOperators.scala index a95b4593fc397..49044c6e24db6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PushedDownOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PushedDownOperators.scala @@ -28,6 +28,7 @@ case class PushedDownOperators( aggregation: Option[Aggregation], sample: Option[TableSampleInfo], limit: Option[Int], + offset: Option[Int], sortValues: Seq[SortOrder], pushedPredicates: Seq[Predicate]) { assert((limit.isEmpty && sortValues.isEmpty) || limit.isDefined) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ScanRelationPushDown.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ScanRelationPushDown.scala index 3a3896d9c9fe8..f69cf937b099b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ScanRelationPushDown.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ScanRelationPushDown.scala @@ -19,12 +19,11 @@ package org.apache.spark.sql.execution.datasources.v2 import scala.collection.mutable -import org.apache.spark.sql.catalyst.expressions.{Alias, AliasHelper, And, Attribute, AttributeReference, Cast, Expression, IntegerLiteral, NamedExpression, PredicateHelper, ProjectionOverSchema, SortOrder, SubqueryExpression} -import org.apache.spark.sql.catalyst.expressions.aggregate +import org.apache.spark.sql.catalyst.expressions.{aggregate, Alias, AliasHelper, And, Attribute, AttributeReference, Cast, Expression, IntegerLiteral, Literal, NamedExpression, PredicateHelper, ProjectionOverSchema, SortOrder, SubqueryExpression} import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression import org.apache.spark.sql.catalyst.optimizer.CollapseProject import org.apache.spark.sql.catalyst.planning.ScanOperation -import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, LeafNode, Limit, LocalLimit, LogicalPlan, Project, Sample, Sort} +import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, LeafNode, Limit, LimitAndOffset, LocalLimit, LogicalPlan, Offset, OffsetAndLimit, Project, Sample, Sort} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.connector.expressions.{SortOrder => V2SortOrder} import org.apache.spark.sql.connector.expressions.aggregate.{Aggregation, Avg, Count, GeneralAggregateFunc, Sum} @@ -32,15 +31,24 @@ import org.apache.spark.sql.connector.expressions.filter.Predicate import org.apache.spark.sql.connector.read.{Scan, ScanBuilder, SupportsPushDownAggregates, SupportsPushDownFilters, V1Scan} import org.apache.spark.sql.execution.datasources.DataSourceStrategy import org.apache.spark.sql.sources -import org.apache.spark.sql.types.{DataType, LongType, StructType} +import org.apache.spark.sql.types.{DataType, IntegerType, LongType, StructType} import org.apache.spark.sql.util.SchemaUtils._ object V2ScanRelationPushDown extends Rule[LogicalPlan] with PredicateHelper with AliasHelper { import DataSourceV2Implicits._ def apply(plan: LogicalPlan): LogicalPlan = { - applyColumnPruning( - applyLimit(pushDownAggregates(pushDownFilters(pushDownSample(createScanBuilder(plan)))))) + val pushdownRules = Seq[LogicalPlan => LogicalPlan] ( + createScanBuilder, + pushDownSample, + pushDownFilters, + pushDownAggregates, + pushDownLimitAndOffset, + pruneColumns) + + pushdownRules.foldLeft(plan) { (newPlan, pushDownRule) => + pushDownRule(newPlan) + } } private def createScanBuilder(plan: LogicalPlan) = plan.transform { @@ -302,7 +310,7 @@ object V2ScanRelationPushDown extends Rule[LogicalPlan] with PredicateHelper wit Cast(expression, expectedDataType) } - def applyColumnPruning(plan: LogicalPlan): LogicalPlan = plan.transform { + def pruneColumns(plan: LogicalPlan): LogicalPlan = plan.transform { case ScanOperation(project, filters, sHolder: ScanBuilderHolder) => // column pruning val normalizedProjects = DataSourceStrategy @@ -372,7 +380,9 @@ object V2ScanRelationPushDown extends Rule[LogicalPlan] with PredicateHelper wit order, project, alwaysInline = true) => val aliasMap = getAliasMap(project) val newOrder = order.map(replaceAlias(_, aliasMap)).asInstanceOf[Seq[SortOrder]] - val orders = DataSourceStrategy.translateSortOrders(newOrder) + val normalizedOrders = DataSourceStrategy.normalizeExprs( + newOrder, sHolder.relation.output).asInstanceOf[Seq[SortOrder]] + val orders = DataSourceStrategy.translateSortOrders(normalizedOrders) if (orders.length == order.length) { val (isPushed, isPartiallyPushed) = PushDownUtils.pushTopN(sHolder.builder, orders.toArray, limit) @@ -396,7 +406,60 @@ object V2ScanRelationPushDown extends Rule[LogicalPlan] with PredicateHelper wit case other => (other, false) } - def applyLimit(plan: LogicalPlan): LogicalPlan = plan.transform { + private def pushDownOffset( + plan: LogicalPlan, + offset: Int): Boolean = plan match { + case sHolder: ScanBuilderHolder => + val isPushed = PushDownUtils.pushOffset(sHolder.builder, offset) + if (isPushed) { + sHolder.pushedOffset = Some(offset) + } + isPushed + case Project(projectList, child) if projectList.forall(_.deterministic) => + pushDownOffset(child, offset) + case _ => false + } + + def pushDownLimitAndOffset(plan: LogicalPlan): LogicalPlan = plan.transform { + case offset @ LimitAndOffset(limit, offsetValue, child) => + val (newChild, canRemoveLimit) = pushDownLimit(child, limit) + if (canRemoveLimit) { + // Try to push down OFFSET only if the LIMIT operator has been pushed and can be removed. + val isPushed = pushDownOffset(newChild, offsetValue) + if (isPushed) { + newChild + } else { + // Keep the OFFSET operator if we failed to push down OFFSET to the data source. + offset.withNewChildren(Seq(newChild)) + } + } else { + // Keep the OFFSET operator if we can't remove LIMIT operator. + offset + } + case globalLimit @ OffsetAndLimit(offset, limit, child) => + // For `df.offset(n).limit(m)`, we can push down `limit(m + n)` first. + val (newChild, canRemoveLimit) = pushDownLimit(child, limit + offset) + if (canRemoveLimit) { + // Try to push down OFFSET only if the LIMIT operator has been pushed and can be removed. + val isPushed = pushDownOffset(newChild, offset) + if (isPushed) { + newChild + } else { + // Still keep the OFFSET operator if we can't push it down. + Offset(Literal(offset), newChild) + } + } else { + // For `df.offset(n).limit(m)`, since we can't push down `limit(m + n)`, + // try to push down `offset(n)` here. + val isPushed = pushDownOffset(child, offset) + if (isPushed) { + // Keep the LIMIT operator if we can't push it down. + Limit(Literal(limit, IntegerType), child) + } else { + // Keep the origin plan if we can't push OFFSET operator and LIMIT operator. + globalLimit + } + } case globalLimit @ Limit(IntegerLiteral(limitValue), child) => val (newChild, canRemoveLimit) = pushDownLimit(child, limitValue) if (canRemoveLimit) { @@ -406,6 +469,13 @@ object V2ScanRelationPushDown extends Rule[LogicalPlan] with PredicateHelper wit globalLimit.child.asInstanceOf[LocalLimit].withNewChildren(Seq(newChild)) globalLimit.withNewChildren(Seq(newLocalLimit)) } + case offset @ Offset(IntegerLiteral(n), child) => + val isPushed = pushDownOffset(child, n) + if (isPushed) { + child + } else { + offset + } } private def getWrappedScan( @@ -420,7 +490,7 @@ object V2ScanRelationPushDown extends Rule[LogicalPlan] with PredicateHelper wit case _ => Array.empty[sources.Filter] } val pushedDownOperators = PushedDownOperators(aggregation, sHolder.pushedSample, - sHolder.pushedLimit, sHolder.sortOrders, sHolder.pushedPredicates) + sHolder.pushedLimit, sHolder.pushedOffset, sHolder.sortOrders, sHolder.pushedPredicates) V1ScanWrapper(v1, pushedFilters, pushedDownOperators) case _ => scan } @@ -433,6 +503,8 @@ case class ScanBuilderHolder( builder: ScanBuilder) extends LeafNode { var pushedLimit: Option[Int] = None + var pushedOffset: Option[Int] = None + var sortOrders: Seq[V2SortOrder] = Seq.empty[V2SortOrder] var pushedSample: Option[TableSampleInfo] = None @@ -440,7 +512,6 @@ case class ScanBuilderHolder( var pushedPredicates: Seq[Predicate] = Seq.empty[Predicate] } - // A wrapper for v1 scan to carry the translated filters and the handled ones, along with // other pushed down operators. This is required by the physical v1 scan node. case class V1ScanWrapper( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCScan.scala index f68f78d51fd96..ea642a3a5e510 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCScan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCScan.scala @@ -18,7 +18,6 @@ package org.apache.spark.sql.execution.datasources.v2.jdbc import org.apache.spark.rdd.RDD import org.apache.spark.sql.{Row, SQLContext} -import org.apache.spark.sql.connector.expressions.SortOrder import org.apache.spark.sql.connector.expressions.filter.Predicate import org.apache.spark.sql.connector.read.V1Scan import org.apache.spark.sql.execution.datasources.jdbc.JDBCRelation @@ -34,7 +33,8 @@ case class JDBCScan( groupByColumns: Option[Array[String]], tableSample: Option[TableSampleInfo], pushedLimit: Int, - sortOrders: Array[SortOrder]) extends V1Scan { + sortOrders: Array[String], + pushedOffset: Int) extends V1Scan { override def readSchema(): StructType = prunedSchema @@ -50,7 +50,7 @@ case class JDBCScan( pushedAggregateColumn } relation.buildScan(columnList, prunedSchema, pushedPredicates, groupByColumns, tableSample, - pushedLimit, sortOrders) + pushedLimit, sortOrders, pushedOffset) } }.asInstanceOf[T] } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCScanBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCScanBuilder.scala index 8b378d2d87c49..cd274e2248529 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCScanBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCScanBuilder.scala @@ -23,7 +23,7 @@ import org.apache.spark.sql.SparkSession import org.apache.spark.sql.connector.expressions.{FieldReference, SortOrder} import org.apache.spark.sql.connector.expressions.aggregate.Aggregation import org.apache.spark.sql.connector.expressions.filter.Predicate -import org.apache.spark.sql.connector.read.{Scan, ScanBuilder, SupportsPushDownAggregates, SupportsPushDownLimit, SupportsPushDownRequiredColumns, SupportsPushDownTableSample, SupportsPushDownTopN, SupportsPushDownV2Filters} +import org.apache.spark.sql.connector.read.{Scan, ScanBuilder, SupportsPushDownAggregates, SupportsPushDownLimit, SupportsPushDownOffset, SupportsPushDownRequiredColumns, SupportsPushDownTableSample, SupportsPushDownTopN, SupportsPushDownV2Filters} import org.apache.spark.sql.execution.datasources.PartitioningUtils import org.apache.spark.sql.execution.datasources.jdbc.{JDBCOptions, JDBCRDD, JDBCRelation} import org.apache.spark.sql.execution.datasources.v2.TableSampleInfo @@ -39,6 +39,7 @@ case class JDBCScanBuilder( with SupportsPushDownRequiredColumns with SupportsPushDownAggregates with SupportsPushDownLimit + with SupportsPushDownOffset with SupportsPushDownTableSample with SupportsPushDownTopN with Logging { @@ -53,7 +54,9 @@ case class JDBCScanBuilder( private var pushedLimit = 0 - private var sortOrders: Array[SortOrder] = Array.empty[SortOrder] + private var pushedOffset = 0 + + private var sortOrders: Array[String] = Array.empty[String] override def pushPredicates(predicates: Array[Predicate]): Array[Predicate] = { if (jdbcOptions.pushDownPredicate) { @@ -138,10 +141,29 @@ case class JDBCScanBuilder( false } + override def pushOffset(offset: Int): Boolean = { + if (jdbcOptions.pushDownOffset && !isPartiallyPushed) { + // Spark pushes down LIMIT first, then OFFSET. In SQL statements, OFFSET is applied before + // LIMIT. Here we need to adjust the LIMIT value to match SQL statements. + if (pushedLimit > 0) { + pushedLimit = pushedLimit - offset + } + pushedOffset = offset + return true + } + false + } + override def pushTopN(orders: Array[SortOrder], limit: Int): Boolean = { if (jdbcOptions.pushDownLimit) { + val dialect = JdbcDialects.get(jdbcOptions.url) + val compiledOrders = orders.flatMap { order => + dialect.compileExpression(order.expression()) + .map(sortKey => s"$sortKey ${order.direction()} ${order.nullOrdering()}") + } + if (orders.length != compiledOrders.length) return false pushedLimit = limit - sortOrders = orders + sortOrders = compiledOrders return true } false @@ -174,6 +196,6 @@ case class JDBCScanBuilder( // prunedSchema and quote them (will become "MAX(SALARY)", "MIN(BONUS)" and can't // be used in sql string. JDBCScan(JDBCRelation(schema, parts, jdbcOptions)(session), finalSchema, pushedPredicate, - pushedAggregateList, pushedGroupBys, tableSample, pushedLimit, sortOrders) + pushedAggregateList, pushedGroupBys, tableSample, pushedLimit, sortOrders, pushedOffset) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala index f2204c0c1e9c3..dbba19002c564 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala @@ -37,16 +37,32 @@ trait LimitExec extends UnaryExecNode { } /** - * Take the first `limit` elements and collect them to a single partition. + * Take the first `limit` elements, collect them to a single partition and then to drop the + * first `offset` elements. * - * This operator will be used when a logical `Limit` operation is the final operator in an - * logical plan, which happens when the user is collecting results back to the driver. + * This operator will be used when a logical `Limit` and/or `Offset` operation is the final operator + * in an logical plan, which happens when the user is collecting results back to the driver. */ -case class CollectLimitExec(limit: Int, offset: Int, child: SparkPlan) extends LimitExec { +case class CollectLimitExec(limit: Int = -1, child: SparkPlan, offset: Int = 0) extends LimitExec { + assert(limit >= 0 || (limit == -1 && offset > 0)) + override def output: Seq[Attribute] = child.output override def outputPartitioning: Partitioning = SinglePartition override def executeCollect(): Array[InternalRow] = { - child.executeTake(limit + offset).drop(offset) + // Because CollectLimitExec collect all the output of child to a single partition, so we need + // collect the first `limit` + `offset` elements and then to drop the first `offset` elements. + // For example: limit is 1 and offset is 2 and the child output two partition. + // The first partition output [1, 2] and the Second partition output [3, 4, 5]. + // Then [1, 2, 3] will be taken and output [3]. + if (limit >= 0) { + if (offset > 0) { + child.executeTake(limit).drop(offset) + } else { + child.executeTake(limit) + } + } else { + child.executeCollect().drop(offset) + } } private val serializer: Serializer = new UnsafeRowSerializer(child.output.size) private lazy val writeMetrics = @@ -62,7 +78,11 @@ case class CollectLimitExec(limit: Int, offset: Int, child: SparkPlan) extends L val singlePartitionRDD = if (childRDD.getNumPartitions == 1) { childRDD } else { - val locallyLimited = child.execute().mapPartitionsInternal(_.take(limit + offset)) + val locallyLimited = if (limit >= 0) { + childRDD.mapPartitionsInternal(_.take(limit)) + } else { + childRDD + } new ShuffledRowRDD( ShuffleExchangeExec.prepareShuffleDependency( locallyLimited, @@ -72,10 +92,25 @@ case class CollectLimitExec(limit: Int, offset: Int, child: SparkPlan) extends L writeMetrics), readMetrics) } - singlePartitionRDD.mapPartitionsInternal(_.drop(offset).take(limit)) + if (limit >= 0) { + if (offset > 0) { + singlePartitionRDD.mapPartitionsInternal(_.slice(offset, limit)) + } else { + singlePartitionRDD.mapPartitionsInternal(_.take(limit)) + } + } else { + singlePartitionRDD.mapPartitionsInternal(_.drop(offset)) + } } } + override def stringArgs: Iterator[Any] = { + super.stringArgs.zipWithIndex.filter { + case (0, 2) => false + case _ => true + }.map(_._1) + } + override protected def withNewChildInternal(newChild: SparkPlan): SparkPlan = copy(child = newChild) } @@ -125,8 +160,8 @@ trait BaseLimitExec extends LimitExec with CodegenSupport { override def outputOrdering: Seq[SortOrder] = child.outputOrdering - protected override def doExecute(): RDD[InternalRow] = child.execute().mapPartitions { iter => - iter.take(limit) + protected override def doExecute(): RDD[InternalRow] = child.execute().mapPartitionsInternal { + iter => iter.take(limit) } override def inputRDDs(): Seq[RDD[InternalRow]] = { @@ -137,10 +172,12 @@ trait BaseLimitExec extends LimitExec with CodegenSupport { // to the parent operator. override def usedInputs: AttributeSet = AttributeSet.empty - lazy val countTerm = BaseLimitExec.newLimitCountTerm() + protected lazy val countTerm = BaseLimitExec.newLimitCountTerm() - override lazy val limitNotReachedChecks: Seq[String] = { + override lazy val limitNotReachedChecks: Seq[String] = if (limit >= 0) { s"$countTerm < $limit" +: super.limitNotReachedChecks + } else { + super.limitNotReachedChecks } protected override def doProduce(ctx: CodegenContext): String = { @@ -174,52 +211,53 @@ case class LocalLimitExec(limit: Int, child: SparkPlan) extends BaseLimitExec { } /** - * Take the first `limit` elements of the child's single output partition. - */ -case class GlobalLimitExec(limit: Int, child: SparkPlan) extends BaseLimitExec { - - override def requiredChildDistribution: List[Distribution] = AllTuples :: Nil - - override protected def withNewChildInternal(newChild: SparkPlan): SparkPlan = - copy(child = newChild) -} - -/** - * Skip the first `offset` elements then take the first `limit` of the following elements in - * the child's single output partition. + * Take the first `limit` elements and then drop the first `offset` elements in the child's single + * output partition. */ -case class GlobalLimitAndOffsetExec( - limit: Int, - offset: Int, - child: SparkPlan) extends BaseLimitExec { +case class GlobalLimitExec(limit: Int = -1, child: SparkPlan, offset: Int = 0) + extends BaseLimitExec { + assert(limit >= 0 || (limit == -1 && offset > 0)) override def requiredChildDistribution: List[Distribution] = AllTuples :: Nil - override def outputPartitioning: Partitioning = child.outputPartitioning - - override def outputOrdering: Seq[SortOrder] = child.outputOrdering - override def doExecute(): RDD[InternalRow] = { - val rdd = child.execute().mapPartitions { iter => iter.take(limit + offset) } - rdd.zipWithIndex().filter(_._2 >= offset).map(_._1) + if (offset > 0) { + if (limit >= 0) { + child.execute().mapPartitionsInternal(iter => iter.slice(offset, limit)) + } else { + child.execute().mapPartitionsInternal(iter => iter.drop(offset)) + } + } else { + super.doExecute() + } } - private lazy val skipTerm = BaseLimitExec.newLimitCountTerm() - override def doConsume(ctx: CodegenContext, input: Seq[ExprCode], row: ExprCode): String = { - // The counter name is already obtained by the upstream operators via `limitNotReachedChecks`. - // Here we have to inline it to not change its name. This is fine as we won't have many limit - // operators in one query. - ctx.addMutableState(CodeGenerator.JAVA_INT, skipTerm, forceInline = true, useFreshName = false) - ctx.addMutableState(CodeGenerator.JAVA_INT, countTerm, forceInline = true, useFreshName = false) - s""" - | if ($skipTerm < $offset) { - | $skipTerm += 1; - | } else if ($countTerm < $limit) { - | $countTerm += 1; - | ${consume(ctx, input)} - | } - """.stripMargin + if (offset > 0) { + val skipTerm = ctx.addMutableState(CodeGenerator.JAVA_INT, "rowsSkipped", forceInline = true) + if (limit > 0) { + // In codegen, we skip the first `offset` rows, then take the first `limit - offset` rows. + val finalLimit = limit - offset + s""" + | if ($skipTerm < $offset) { + | $skipTerm += 1; + | } else if ($countTerm < $finalLimit) { + | $countTerm += 1; + | ${consume(ctx, input)} + | } + """.stripMargin + } else { + s""" + | if ($skipTerm < $offset) { + | $skipTerm += 1; + | } else { + | ${consume(ctx, input)} + | } + """.stripMargin + } + } else { + super.doConsume(ctx, input, row) + } } override protected def withNewChildInternal(newChild: SparkPlan): SparkPlan = @@ -227,18 +265,18 @@ case class GlobalLimitAndOffsetExec( } /** - * Take the first limit elements as defined by the sortOrder, and do projection if needed. - * This is logically equivalent to having a Limit operator after a [[SortExec]] operator, - * or having a [[ProjectExec]] operator between them. + * Take the first `limit` elements as defined by the sortOrder, then drop the first `offset` + * elements, and do projection if needed. This is logically equivalent to having a Limit and/or + * Offset operator after a [[SortExec]] operator, or having a [[ProjectExec]] operator between them. * This could have been named TopK, but Spark's top operator does the opposite in ordering * so we name it TakeOrdered to avoid confusion. */ case class TakeOrderedAndProjectExec( limit: Int, - offset: Int, sortOrder: Seq[SortOrder], projectList: Seq[NamedExpression], - child: SparkPlan) extends UnaryExecNode { + child: SparkPlan, + offset: Int = 0) extends UnaryExecNode { override def output: Seq[Attribute] = { projectList.map(_.toAttribute) @@ -246,7 +284,8 @@ case class TakeOrderedAndProjectExec( override def executeCollect(): Array[InternalRow] = { val ord = new LazilyGeneratedOrdering(sortOrder, child.output) - val data = child.execute().map(_.copy()).takeOrdered(limit + offset)(ord).drop(offset) + val limited = child.execute().mapPartitionsInternal(_.map(_.copy())).takeOrdered(limit)(ord) + val data = if (offset > 0) limited.drop(offset) else limited if (projectList != child.output) { val proj = UnsafeProjection.create(projectList, child.output) data.map(r => proj(r).copy()) @@ -272,9 +311,10 @@ case class TakeOrderedAndProjectExec( val singlePartitionRDD = if (childRDD.getNumPartitions == 1) { childRDD } else { - val localTopK = childRDD.mapPartitions { iter => + val localTopK = childRDD.mapPartitionsInternal { iter => Utils.takeOrdered(iter.map(_.copy()), limit)(ord) } + new ShuffledRowRDD( ShuffleExchangeExec.prepareShuffleDependency( localTopK, @@ -284,8 +324,9 @@ case class TakeOrderedAndProjectExec( writeMetrics), readMetrics) } - singlePartitionRDD.mapPartitions { iter => - val topK = Utils.takeOrdered(iter.map(_.copy()), limit)(ord) + singlePartitionRDD.mapPartitionsInternal { iter => + val limited = Utils.takeOrdered(iter.map(_.copy()), limit)(ord) + val topK = if (offset > 0) limited.drop(offset) else limited if (projectList != child.output) { val proj = UnsafeProjection.create(projectList, child.output) topK.map(r => proj(r)) @@ -307,6 +348,13 @@ case class TakeOrderedAndProjectExec( s"TakeOrderedAndProject(limit=$limit, orderBy=$orderByString, output=$outputString)" } + override def stringArgs: Iterator[Any] = { + super.stringArgs.zipWithIndex.filter { + case (0, 4) => false + case _ => true + }.map(_._1) + } + override protected def withNewChildInternal(newChild: SparkPlan): SparkPlan = copy(child = newChild) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/H2Dialect.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/H2Dialect.scala index dc834f1d432b1..db24267d4f3fe 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/H2Dialect.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/H2Dialect.scala @@ -35,8 +35,10 @@ private[sql] object H2Dialect extends JdbcDialect { url.toLowerCase(Locale.ROOT).startsWith("jdbc:h2") private val supportedFunctions = - Set("ABS", "COALESCE", "LN", "EXP", "POWER", "SQRT", "FLOOR", "CEIL", - "SUBSTRING", "UPPER", "LOWER", "TRANSLATE", "TRIM") + Set("ABS", "COALESCE", "GREATEST", "LEAST", "RAND", "LOG", "LOG10", "LN", "EXP", + "POWER", "SQRT", "FLOOR", "CEIL", "ROUND", "SIN", "SINH", "COS", "COSH", "TAN", + "TANH", "COT", "ASIN", "ACOS", "ATAN", "ATAN2", "DEGREES", "RADIANS", "SIGN", + "PI", "SUBSTRING", "UPPER", "LOWER", "TRANSLATE", "TRIM") override def isSupportedFunction(funcName: String): Boolean = supportedFunctions.contains(funcName) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcDialects.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcDialects.scala index 8115ec292f8d4..d42d4e8fc0bac 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcDialects.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcDialects.scala @@ -549,6 +549,13 @@ abstract class JdbcDialect extends Serializable with Logging{ if (limit > 0 ) s"LIMIT $limit" else "" } + /** + * returns the OFFSET clause for the SELECT statement + */ + def getOffsetClause(offset: Integer): String = { + if (offset > 0 ) s"OFFSET $offset" else "" + } + def supportsTableSample: Boolean = false def getTableSample(sample: TableSampleInfo): String = diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/limit.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/limit.sql index bc0b5d6dddc52..f59575817d6b8 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/limit.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/limit.sql @@ -12,25 +12,24 @@ SELECT '' AS five, unique1, unique2, stringu1 SELECT '' AS two, unique1, unique2, stringu1 FROM onek WHERE unique1 > 60 AND unique1 < 63 ORDER BY unique1 LIMIT 5; --- [SPARK-28330] ANSI SQL: Top-level in --- SELECT '' AS three, unique1, unique2, stringu1 --- FROM onek WHERE unique1 > 100 --- ORDER BY unique1 LIMIT 3 OFFSET 20; --- SELECT '' AS zero, unique1, unique2, stringu1 --- FROM onek WHERE unique1 < 50 --- ORDER BY unique1 DESC LIMIT 8 OFFSET 99; --- SELECT '' AS eleven, unique1, unique2, stringu1 --- FROM onek WHERE unique1 < 50 --- ORDER BY unique1 DESC LIMIT 20 OFFSET 39; --- SELECT '' AS ten, unique1, unique2, stringu1 --- FROM onek --- ORDER BY unique1 OFFSET 990; +SELECT '' AS three, unique1, unique2, stringu1 + FROM onek WHERE unique1 > 100 + ORDER BY unique1 LIMIT 3 OFFSET 20; +SELECT '' AS zero, unique1, unique2, stringu1 + FROM onek WHERE unique1 < 50 + ORDER BY unique1 DESC LIMIT 8 OFFSET 99; +SELECT '' AS eleven, unique1, unique2, stringu1 + FROM onek WHERE unique1 < 50 + ORDER BY unique1 DESC LIMIT 20 OFFSET 39; +SELECT '' AS ten, unique1, unique2, stringu1 + FROM onek + ORDER BY unique1 OFFSET 990; -- SELECT '' AS five, unique1, unique2, stringu1 -- FROM onek -- ORDER BY unique1 OFFSET 990 LIMIT 5; --- SELECT '' AS five, unique1, unique2, stringu1 --- FROM onek --- ORDER BY unique1 LIMIT 5 OFFSET 900; +SELECT '' AS five, unique1, unique2, stringu1 + FROM onek + ORDER BY unique1 LIMIT 5 OFFSET 900; CREATE OR REPLACE TEMPORARY VIEW INT8_TBL AS SELECT * FROM (VALUES @@ -45,8 +44,7 @@ CREATE OR REPLACE TEMPORARY VIEW INT8_TBL AS SELECT * FROM -- constant, so to ensure executor is exercised, do this: -- [SPARK-29650] Discard a NULL constant in LIMIT select * from int8_tbl limit (case when random() < 0.5 then bigint(null) end); --- [SPARK-28330] ANSI SQL: Top-level in --- select * from int8_tbl offset (case when random() < 0.5 then bigint(null) end); +select * from int8_tbl offset (case when random() < 0.5 then bigint(null) end); -- Test assorted cases involving backwards fetch from a LIMIT plan node -- [SPARK-20965] Support PREPARE/EXECUTE/DECLARE/FETCH statements @@ -90,7 +88,7 @@ DROP VIEW INT8_TBL; -- Stress test for variable LIMIT in conjunction with bounded-heap sorting --- [SPARK-28330] ANSI SQL: Top-level in +-- [SPARK-27767] Built-in function: generate_series -- SELECT -- (SELECT n -- FROM (VALUES (1)) AS x, diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/exists-subquery/exists-orderby-limit.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/exists-subquery/exists-orderby-limit.sql index 580fc1d4162eb..8e6b49fea8f4e 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/subquery/exists-subquery/exists-orderby-limit.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/exists-subquery/exists-orderby-limit.sql @@ -121,3 +121,81 @@ WHERE NOT EXISTS (SELECT max(dept.dept_id) WHERE dept.dept_id > 100 GROUP BY state LIMIT 1); + +-- limit and offset in the exists subquery block. +-- TC.03.01 +SELECT * +FROM emp +WHERE EXISTS (SELECT dept.dept_name + FROM dept + WHERE dept.dept_id > 10 + LIMIT 1 + OFFSET 2); + +-- limit and offset in the exists subquery block with aggregate. +-- TC.03.02 +SELECT * +FROM emp +WHERE EXISTS (SELECT max(dept.dept_id) + FROM dept + GROUP BY state + LIMIT 1 + OFFSET 2); + +-- limit and offset in the not exists subquery block. +-- TC.03.03 +SELECT * +FROM emp +WHERE NOT EXISTS (SELECT dept.dept_name + FROM dept + WHERE dept.dept_id > 100 + LIMIT 1 + OFFSET 2); + +-- limit and offset in the not exists subquery block with aggregates. +-- TC.03.04 +SELECT * +FROM emp +WHERE NOT EXISTS (SELECT max(dept.dept_id) + FROM dept + WHERE dept.dept_id > 100 + GROUP BY state + LIMIT 1 + OFFSET 2); + +-- offset in the exists subquery block. +-- TC.04.01 +SELECT * +FROM emp +WHERE EXISTS (SELECT dept.dept_name + FROM dept + WHERE dept.dept_id > 10 + OFFSET 2); + +-- offset in the exists subquery block with aggregate. +-- TC.04.02 +SELECT * +FROM emp +WHERE EXISTS (SELECT max(dept.dept_id) + FROM dept + GROUP BY state + OFFSET 2); + +-- limit in the not exists subquery block. +-- TC.04.03 +SELECT * +FROM emp +WHERE NOT EXISTS (SELECT dept.dept_name + FROM dept + WHERE dept.dept_id > 100 + OFFSET 2); + +-- limit in the not exists subquery block with aggregates. +-- TC.04.04 +SELECT * +FROM emp +WHERE NOT EXISTS (SELECT max(dept.dept_id) + FROM dept + WHERE dept.dept_id > 100 + GROUP BY state + OFFSET 2); diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-limit.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-limit.sql index 53fc2b8be7501..b9b062a5f7a1d 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-limit.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-limit.sql @@ -100,4 +100,113 @@ WHERE t1d NOT IN (SELECT t2d LIMIT 1) GROUP BY t1b ORDER BY t1b NULLS last -LIMIT 1; \ No newline at end of file +LIMIT 1; + +-- LIMIT and OFFSET in parent side +-- TC 02.01 +SELECT * +FROM t1 +WHERE t1a IN (SELECT t2a + FROM t2 + WHERE t1d = t2d) +LIMIT 2 +OFFSET 2; + +-- TC 02.02 +SELECT * +FROM t1 +WHERE t1c IN (SELECT t2c + FROM t2 + WHERE t2b >= 8 + LIMIT 2 + OFFSET 2) +LIMIT 4 +OFFSET 2; + +-- TC 02.03 +SELECT Count(DISTINCT( t1a )), + t1b +FROM t1 +WHERE t1d IN (SELECT t2d + FROM t2 + ORDER BY t2c, t2d + LIMIT 2) +GROUP BY t1b +ORDER BY t1b DESC NULLS FIRST +LIMIT 1 +OFFSET 1; + +-- LIMIT with NOT IN +-- TC 02.04 +SELECT * +FROM t1 +WHERE t1b NOT IN (SELECT t2b + FROM t2 + WHERE t2b > 6 + LIMIT 2 + OFFSET 2); + +-- TC 02.05 +SELECT Count(DISTINCT( t1a )), + t1b +FROM t1 +WHERE t1d NOT IN (SELECT t2d + FROM t2 + ORDER BY t2b DESC nulls first, t2d + LIMIT 1 + OFFSET 1) +GROUP BY t1b +ORDER BY t1b NULLS last +LIMIT 1 +OFFSET 1; + +-- OFFSET in parent side +-- TC 03.01 +SELECT * +FROM t1 +WHERE t1a IN (SELECT t2a + FROM t2 + WHERE t1d = t2d) +OFFSET 2; + +-- TC 03.02 +SELECT * +FROM t1 +WHERE t1c IN (SELECT t2c + FROM t2 + WHERE t2b >= 8 + OFFSET 2) +OFFSET 4; + +-- TC 03.03 +SELECT Count(DISTINCT( t1a )), + t1b +FROM t1 +WHERE t1d IN (SELECT t2d + FROM t2 + ORDER BY t2c, t2d + OFFSET 2) +GROUP BY t1b +ORDER BY t1b DESC NULLS FIRST +OFFSET 1; + +-- OFFSET with NOT IN +-- TC 03.04 +SELECT * +FROM t1 +WHERE t1b NOT IN (SELECT t2b + FROM t2 + WHERE t2b > 6 + OFFSET 2); + +-- TC 03.05 +SELECT Count(DISTINCT( t1a )), + t1b +FROM t1 +WHERE t1d NOT IN (SELECT t2d + FROM t2 + ORDER BY t2b DESC nulls first, t2d + OFFSET 1) +GROUP BY t1b +ORDER BY t1b NULLS last +OFFSET 1; diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/limit.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/limit.sql.out index 2384010c67b4d..d6526d77d3cad 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/limit.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/limit.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 7 +-- Number of queries: 13 -- !query @@ -38,6 +38,81 @@ struct 62 633 KCAAAA +-- !query +SELECT '' AS three, unique1, unique2, stringu1 + FROM onek WHERE unique1 > 100 + ORDER BY unique1 LIMIT 3 OFFSET 20 +-- !query schema +struct +-- !query output + 121 700 REAAAA + 122 519 SEAAAA + 123 777 TEAAAA + + +-- !query +SELECT '' AS zero, unique1, unique2, stringu1 + FROM onek WHERE unique1 < 50 + ORDER BY unique1 DESC LIMIT 8 OFFSET 99 +-- !query schema +struct +-- !query output + + + +-- !query +SELECT '' AS eleven, unique1, unique2, stringu1 + FROM onek WHERE unique1 < 50 + ORDER BY unique1 DESC LIMIT 20 OFFSET 39 +-- !query schema +struct +-- !query output + 10 520 KAAAAA + 9 49 JAAAAA + 8 653 IAAAAA + 7 647 HAAAAA + 6 978 GAAAAA + 5 541 FAAAAA + 4 833 EAAAAA + 3 431 DAAAAA + 2 326 CAAAAA + 1 214 BAAAAA + 0 998 AAAAAA + + +-- !query +SELECT '' AS ten, unique1, unique2, stringu1 + FROM onek + ORDER BY unique1 OFFSET 990 +-- !query schema +struct +-- !query output + 990 369 CMAAAA + 991 426 DMAAAA + 992 363 EMAAAA + 993 661 FMAAAA + 994 695 GMAAAA + 995 144 HMAAAA + 996 258 IMAAAA + 997 21 JMAAAA + 998 549 KMAAAA + 999 152 LMAAAA + + +-- !query +SELECT '' AS five, unique1, unique2, stringu1 + FROM onek + ORDER BY unique1 LIMIT 5 OFFSET 900 +-- !query schema +struct +-- !query output + 900 913 QIAAAA + 901 931 RIAAAA + 902 702 SIAAAA + 903 641 TIAAAA + 904 793 UIAAAA + + -- !query CREATE OR REPLACE TEMPORARY VIEW INT8_TBL AS SELECT * FROM (VALUES @@ -62,6 +137,15 @@ org.apache.spark.sql.AnalysisException The limit expression must evaluate to a constant value, but got CASE WHEN (_nondeterministic < CAST(0.5BD AS DOUBLE)) THEN CAST(NULL AS BIGINT) END +-- !query +select * from int8_tbl offset (case when random() < 0.5 then bigint(null) end) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +The offset expression must evaluate to a constant value, but got CASE WHEN (_nondeterministic < CAST(0.5BD AS DOUBLE)) THEN CAST(NULL AS BIGINT) END + + -- !query DROP VIEW INT8_TBL -- !query schema diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/exists-subquery/exists-orderby-limit.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/exists-subquery/exists-orderby-limit.sql.out index ebd4da6ccbd5d..3442cfd689a2b 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/exists-subquery/exists-orderby-limit.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/exists-subquery/exists-orderby-limit.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 12 +-- Number of queries: 20 -- !query @@ -220,3 +220,177 @@ struct 600 emp 6 - no dept 2001-01-01 400.0 100 700 emp 7 2010-01-01 400.0 100 800 emp 8 2016-01-01 150.0 70 + + +-- !query +SELECT * +FROM emp +WHERE EXISTS (SELECT dept.dept_name + FROM dept + WHERE dept.dept_id > 10 + LIMIT 1 + OFFSET 2) +-- !query schema +struct +-- !query output +100 emp 1 2005-01-01 100.0 10 +100 emp 1 2005-01-01 100.0 10 +200 emp 2 2003-01-01 200.0 10 +300 emp 3 2002-01-01 300.0 20 +400 emp 4 2005-01-01 400.0 30 +500 emp 5 2001-01-01 400.0 NULL +600 emp 6 - no dept 2001-01-01 400.0 100 +700 emp 7 2010-01-01 400.0 100 +800 emp 8 2016-01-01 150.0 70 + + +-- !query +SELECT * +FROM emp +WHERE EXISTS (SELECT max(dept.dept_id) + FROM dept + GROUP BY state + LIMIT 1 + OFFSET 2) +-- !query schema +struct +-- !query output +100 emp 1 2005-01-01 100.0 10 +100 emp 1 2005-01-01 100.0 10 +200 emp 2 2003-01-01 200.0 10 +300 emp 3 2002-01-01 300.0 20 +400 emp 4 2005-01-01 400.0 30 +500 emp 5 2001-01-01 400.0 NULL +600 emp 6 - no dept 2001-01-01 400.0 100 +700 emp 7 2010-01-01 400.0 100 +800 emp 8 2016-01-01 150.0 70 + + +-- !query +SELECT * +FROM emp +WHERE NOT EXISTS (SELECT dept.dept_name + FROM dept + WHERE dept.dept_id > 100 + LIMIT 1 + OFFSET 2) +-- !query schema +struct +-- !query output +100 emp 1 2005-01-01 100.0 10 +100 emp 1 2005-01-01 100.0 10 +200 emp 2 2003-01-01 200.0 10 +300 emp 3 2002-01-01 300.0 20 +400 emp 4 2005-01-01 400.0 30 +500 emp 5 2001-01-01 400.0 NULL +600 emp 6 - no dept 2001-01-01 400.0 100 +700 emp 7 2010-01-01 400.0 100 +800 emp 8 2016-01-01 150.0 70 + + +-- !query +SELECT * +FROM emp +WHERE NOT EXISTS (SELECT max(dept.dept_id) + FROM dept + WHERE dept.dept_id > 100 + GROUP BY state + LIMIT 1 + OFFSET 2) +-- !query schema +struct +-- !query output +100 emp 1 2005-01-01 100.0 10 +100 emp 1 2005-01-01 100.0 10 +200 emp 2 2003-01-01 200.0 10 +300 emp 3 2002-01-01 300.0 20 +400 emp 4 2005-01-01 400.0 30 +500 emp 5 2001-01-01 400.0 NULL +600 emp 6 - no dept 2001-01-01 400.0 100 +700 emp 7 2010-01-01 400.0 100 +800 emp 8 2016-01-01 150.0 70 + + +-- !query +SELECT * +FROM emp +WHERE EXISTS (SELECT dept.dept_name + FROM dept + WHERE dept.dept_id > 10 + OFFSET 2) +-- !query schema +struct +-- !query output +100 emp 1 2005-01-01 100.0 10 +100 emp 1 2005-01-01 100.0 10 +200 emp 2 2003-01-01 200.0 10 +300 emp 3 2002-01-01 300.0 20 +400 emp 4 2005-01-01 400.0 30 +500 emp 5 2001-01-01 400.0 NULL +600 emp 6 - no dept 2001-01-01 400.0 100 +700 emp 7 2010-01-01 400.0 100 +800 emp 8 2016-01-01 150.0 70 + + +-- !query +SELECT * +FROM emp +WHERE EXISTS (SELECT max(dept.dept_id) + FROM dept + GROUP BY state + OFFSET 2) +-- !query schema +struct +-- !query output +100 emp 1 2005-01-01 100.0 10 +100 emp 1 2005-01-01 100.0 10 +200 emp 2 2003-01-01 200.0 10 +300 emp 3 2002-01-01 300.0 20 +400 emp 4 2005-01-01 400.0 30 +500 emp 5 2001-01-01 400.0 NULL +600 emp 6 - no dept 2001-01-01 400.0 100 +700 emp 7 2010-01-01 400.0 100 +800 emp 8 2016-01-01 150.0 70 + + +-- !query +SELECT * +FROM emp +WHERE NOT EXISTS (SELECT dept.dept_name + FROM dept + WHERE dept.dept_id > 100 + OFFSET 2) +-- !query schema +struct +-- !query output +100 emp 1 2005-01-01 100.0 10 +100 emp 1 2005-01-01 100.0 10 +200 emp 2 2003-01-01 200.0 10 +300 emp 3 2002-01-01 300.0 20 +400 emp 4 2005-01-01 400.0 30 +500 emp 5 2001-01-01 400.0 NULL +600 emp 6 - no dept 2001-01-01 400.0 100 +700 emp 7 2010-01-01 400.0 100 +800 emp 8 2016-01-01 150.0 70 + + +-- !query +SELECT * +FROM emp +WHERE NOT EXISTS (SELECT max(dept.dept_id) + FROM dept + WHERE dept.dept_id > 100 + GROUP BY state + OFFSET 2) +-- !query schema +struct +-- !query output +100 emp 1 2005-01-01 100.0 10 +100 emp 1 2005-01-01 100.0 10 +200 emp 2 2003-01-01 200.0 10 +300 emp 3 2002-01-01 300.0 20 +400 emp 4 2005-01-01 400.0 30 +500 emp 5 2001-01-01 400.0 NULL +600 emp 6 - no dept 2001-01-01 400.0 100 +700 emp 7 2010-01-01 400.0 100 +800 emp 8 2016-01-01 150.0 70 diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-limit.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-limit.sql.out index e24538b9138ba..08f76f0936a56 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-limit.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-limit.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 8 +-- Number of queries: 18 -- !query @@ -145,3 +145,175 @@ LIMIT 1 struct -- !query output 1 6 + + +-- !query +SELECT * +FROM t1 +WHERE t1a IN (SELECT t2a + FROM t2 + WHERE t1d = t2d) +LIMIT 2 +OFFSET 2 +-- !query schema +struct +-- !query output +val1e 10 NULL 19 17.0 25.0 2600 2014-05-04 01:01:00 2014-05-04 +val1e 10 NULL 19 17.0 25.0 2600 2014-09-04 01:02:00.001 2014-09-04 + + +-- !query +SELECT * +FROM t1 +WHERE t1c IN (SELECT t2c + FROM t2 + WHERE t2b >= 8 + LIMIT 2 + OFFSET 2) +LIMIT 4 +OFFSET 2 +-- !query schema +struct +-- !query output +val1d NULL 16 19 17.0 25.0 2600 2014-07-04 01:02:00.001 NULL +val1d NULL 16 22 17.0 25.0 2600 2014-06-04 01:01:00 NULL + + +-- !query +SELECT Count(DISTINCT( t1a )), + t1b +FROM t1 +WHERE t1d IN (SELECT t2d + FROM t2 + ORDER BY t2c, t2d + LIMIT 2) +GROUP BY t1b +ORDER BY t1b DESC NULLS FIRST +LIMIT 1 +OFFSET 1 +-- !query schema +struct +-- !query output +1 10 + + +-- !query +SELECT * +FROM t1 +WHERE t1b NOT IN (SELECT t2b + FROM t2 + WHERE t2b > 6 + LIMIT 2 + OFFSET 2) +-- !query schema +struct +-- !query output +val1a 16 12 10 15.0 20.0 2000 2014-07-04 01:01:00 2014-07-04 +val1a 16 12 21 15.0 20.0 2000 2014-06-04 01:02:00.001 2014-06-04 +val1a 6 8 10 15.0 20.0 2000 2014-04-04 01:00:00 2014-04-04 +val1a 6 8 10 15.0 20.0 2000 2014-04-04 01:02:00.001 2014-04-04 +val1d 10 NULL 12 17.0 25.0 2600 2015-05-04 01:01:00 2015-05-04 +val1e 10 NULL 19 17.0 25.0 2600 2014-05-04 01:01:00 2014-05-04 +val1e 10 NULL 19 17.0 25.0 2600 2014-09-04 01:02:00.001 2014-09-04 +val1e 10 NULL 25 17.0 25.0 2600 2014-08-04 01:01:00 2014-08-04 + + +-- !query +SELECT Count(DISTINCT( t1a )), + t1b +FROM t1 +WHERE t1d NOT IN (SELECT t2d + FROM t2 + ORDER BY t2b DESC nulls first, t2d + LIMIT 1 + OFFSET 1) +GROUP BY t1b +ORDER BY t1b NULLS last +LIMIT 1 +OFFSET 1 +-- !query schema +struct +-- !query output +2 8 + + +-- !query +SELECT * +FROM t1 +WHERE t1a IN (SELECT t2a + FROM t2 + WHERE t1d = t2d) +OFFSET 2 +-- !query schema +struct +-- !query output +val1e 10 NULL 19 17.0 25.0 2600 2014-05-04 01:01:00 2014-05-04 +val1e 10 NULL 19 17.0 25.0 2600 2014-09-04 01:02:00.001 2014-09-04 + + +-- !query +SELECT * +FROM t1 +WHERE t1c IN (SELECT t2c + FROM t2 + WHERE t2b >= 8 + OFFSET 2) +OFFSET 4 +-- !query schema +struct +-- !query output +val1d NULL 16 19 17.0 25.0 2600 2014-07-04 01:02:00.001 NULL +val1d NULL 16 22 17.0 25.0 2600 2014-06-04 01:01:00 NULL + + +-- !query +SELECT Count(DISTINCT( t1a )), + t1b +FROM t1 +WHERE t1d IN (SELECT t2d + FROM t2 + ORDER BY t2c, t2d + OFFSET 2) +GROUP BY t1b +ORDER BY t1b DESC NULLS FIRST +OFFSET 1 +-- !query schema +struct +-- !query output +1 10 +2 8 + + +-- !query +SELECT * +FROM t1 +WHERE t1b NOT IN (SELECT t2b + FROM t2 + WHERE t2b > 6 + OFFSET 2) +-- !query schema +struct +-- !query output +val1a 16 12 10 15.0 20.0 2000 2014-07-04 01:01:00 2014-07-04 +val1a 16 12 21 15.0 20.0 2000 2014-06-04 01:02:00.001 2014-06-04 +val1a 6 8 10 15.0 20.0 2000 2014-04-04 01:00:00 2014-04-04 +val1a 6 8 10 15.0 20.0 2000 2014-04-04 01:02:00.001 2014-04-04 + + +-- !query +SELECT Count(DISTINCT( t1a )), + t1b +FROM t1 +WHERE t1d NOT IN (SELECT t2d + FROM t2 + ORDER BY t2b DESC nulls first, t2d + OFFSET 1) +GROUP BY t1b +ORDER BY t1b NULLS last +OFFSET 1 +-- !query schema +struct +-- !query output +2 10 +1 16 +1 NULL diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q10.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q10.sf100/explain.txt index 997ec28f49296..02ff6fcce531d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q10.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q10.sf100/explain.txt @@ -262,7 +262,7 @@ Results [14]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, count (48) TakeOrderedAndProject Input [14]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cnt1#37, cd_purchase_estimate#28, cnt2#38, cd_credit_rating#29, cnt3#39, cd_dep_count#30, cnt4#40, cd_dep_employed_count#31, cnt5#41, cd_dep_college_count#32, cnt6#42] -Arguments: 100, 0, [cd_gender#25 ASC NULLS FIRST, cd_marital_status#26 ASC NULLS FIRST, cd_education_status#27 ASC NULLS FIRST, cd_purchase_estimate#28 ASC NULLS FIRST, cd_credit_rating#29 ASC NULLS FIRST, cd_dep_count#30 ASC NULLS FIRST, cd_dep_employed_count#31 ASC NULLS FIRST, cd_dep_college_count#32 ASC NULLS FIRST], [cd_gender#25, cd_marital_status#26, cd_education_status#27, cnt1#37, cd_purchase_estimate#28, cnt2#38, cd_credit_rating#29, cnt3#39, cd_dep_count#30, cnt4#40, cd_dep_employed_count#31, cnt5#41, cd_dep_college_count#32, cnt6#42] +Arguments: 100, [cd_gender#25 ASC NULLS FIRST, cd_marital_status#26 ASC NULLS FIRST, cd_education_status#27 ASC NULLS FIRST, cd_purchase_estimate#28 ASC NULLS FIRST, cd_credit_rating#29 ASC NULLS FIRST, cd_dep_count#30 ASC NULLS FIRST, cd_dep_employed_count#31 ASC NULLS FIRST, cd_dep_college_count#32 ASC NULLS FIRST], [cd_gender#25, cd_marital_status#26, cd_education_status#27, cnt1#37, cd_purchase_estimate#28, cnt2#38, cd_credit_rating#29, cnt3#39, cd_dep_count#30, cnt4#40, cd_dep_employed_count#31, cnt5#41, cd_dep_college_count#32, cnt6#42] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q10/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q10/explain.txt index d0fa367ac3724..a565ac5010026 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q10/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q10/explain.txt @@ -242,7 +242,7 @@ Results [14]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, count (44) TakeOrderedAndProject Input [14]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cnt1#36, cd_purchase_estimate#26, cnt2#37, cd_credit_rating#27, cnt3#38, cd_dep_count#28, cnt4#39, cd_dep_employed_count#29, cnt5#40, cd_dep_college_count#30, cnt6#41] -Arguments: 100, 0, [cd_gender#23 ASC NULLS FIRST, cd_marital_status#24 ASC NULLS FIRST, cd_education_status#25 ASC NULLS FIRST, cd_purchase_estimate#26 ASC NULLS FIRST, cd_credit_rating#27 ASC NULLS FIRST, cd_dep_count#28 ASC NULLS FIRST, cd_dep_employed_count#29 ASC NULLS FIRST, cd_dep_college_count#30 ASC NULLS FIRST], [cd_gender#23, cd_marital_status#24, cd_education_status#25, cnt1#36, cd_purchase_estimate#26, cnt2#37, cd_credit_rating#27, cnt3#38, cd_dep_count#28, cnt4#39, cd_dep_employed_count#29, cnt5#40, cd_dep_college_count#30, cnt6#41] +Arguments: 100, [cd_gender#23 ASC NULLS FIRST, cd_marital_status#24 ASC NULLS FIRST, cd_education_status#25 ASC NULLS FIRST, cd_purchase_estimate#26 ASC NULLS FIRST, cd_credit_rating#27 ASC NULLS FIRST, cd_dep_count#28 ASC NULLS FIRST, cd_dep_employed_count#29 ASC NULLS FIRST, cd_dep_college_count#30 ASC NULLS FIRST], [cd_gender#23, cd_marital_status#24, cd_education_status#25, cnt1#36, cd_purchase_estimate#26, cnt2#37, cd_credit_rating#27, cnt3#38, cd_dep_count#28, cnt4#39, cd_dep_employed_count#29, cnt5#40, cd_dep_college_count#30, cnt6#41] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q19.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q19.sf100/explain.txt index 12fec29da2e1c..6b4fce45b44f2 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q19.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q19.sf100/explain.txt @@ -218,7 +218,7 @@ Results [5]: [i_brand_id#21 AS brand_id#31, i_brand#22 AS brand#32, i_manufact_i (39) TakeOrderedAndProject Input [5]: [brand_id#31, brand#32, i_manufact_id#23, i_manufact#24, ext_price#33] -Arguments: 100, 0, [ext_price#33 DESC NULLS LAST, brand#32 ASC NULLS FIRST, brand_id#31 ASC NULLS FIRST, i_manufact_id#23 ASC NULLS FIRST, i_manufact#24 ASC NULLS FIRST], [brand_id#31, brand#32, i_manufact_id#23, i_manufact#24, ext_price#33] +Arguments: 100, [ext_price#33 DESC NULLS LAST, brand#32 ASC NULLS FIRST, brand_id#31 ASC NULLS FIRST, i_manufact_id#23 ASC NULLS FIRST, i_manufact#24 ASC NULLS FIRST], [brand_id#31, brand#32, i_manufact_id#23, i_manufact#24, ext_price#33] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q19/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q19/explain.txt index c70751cf80f87..e58fb8a1e1364 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q19/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q19/explain.txt @@ -218,5 +218,5 @@ Results [5]: [i_brand_id#11 AS brand_id#30, i_brand#12 AS brand#31, i_manufact_i (39) TakeOrderedAndProject Input [5]: [brand_id#30, brand#31, i_manufact_id#13, i_manufact#14, ext_price#32] -Arguments: 100, 0, [ext_price#32 DESC NULLS LAST, brand#31 ASC NULLS FIRST, brand_id#30 ASC NULLS FIRST, i_manufact_id#13 ASC NULLS FIRST, i_manufact#14 ASC NULLS FIRST], [brand_id#30, brand#31, i_manufact_id#13, i_manufact#14, ext_price#32] +Arguments: 100, [ext_price#32 DESC NULLS LAST, brand#31 ASC NULLS FIRST, brand_id#30 ASC NULLS FIRST, i_manufact_id#13 ASC NULLS FIRST, i_manufact#14 ASC NULLS FIRST], [brand_id#30, brand#31, i_manufact_id#13, i_manufact#14, ext_price#32] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q27.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q27.sf100/explain.txt index b24f36d19dd9b..65572cfd770d0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q27.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q27.sf100/explain.txt @@ -404,7 +404,7 @@ Results [7]: [null AS i_item_id#102, null AS s_state#103, 1 AS g_state#104, avg( (73) TakeOrderedAndProject Input [7]: [i_item_id#20, s_state#17, g_state#47, agg1#48, agg2#49, agg3#50, agg4#51] -Arguments: 100, 0, [i_item_id#20 ASC NULLS FIRST, s_state#17 ASC NULLS FIRST], [i_item_id#20, s_state#17, g_state#47, agg1#48, agg2#49, agg3#50, agg4#51] +Arguments: 100, [i_item_id#20 ASC NULLS FIRST, s_state#17 ASC NULLS FIRST], [i_item_id#20, s_state#17, g_state#47, agg1#48, agg2#49, agg3#50, agg4#51] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q27/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q27/explain.txt index 5fee0b8192a96..0fe719015aaa3 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q27/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q27/explain.txt @@ -404,7 +404,7 @@ Results [7]: [null AS i_item_id#102, null AS s_state#103, 1 AS g_state#104, avg( (73) TakeOrderedAndProject Input [7]: [i_item_id#20, s_state#17, g_state#47, agg1#48, agg2#49, agg3#50, agg4#51] -Arguments: 100, 0, [i_item_id#20 ASC NULLS FIRST, s_state#17 ASC NULLS FIRST], [i_item_id#20, s_state#17, g_state#47, agg1#48, agg2#49, agg3#50, agg4#51] +Arguments: 100, [i_item_id#20 ASC NULLS FIRST, s_state#17 ASC NULLS FIRST], [i_item_id#20, s_state#17, g_state#47, agg1#48, agg2#49, agg3#50, agg4#51] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q3.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q3.sf100/explain.txt index 6a6ae985d1f65..36258b96121f4 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q3.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q3.sf100/explain.txt @@ -96,7 +96,7 @@ Results [4]: [d_year#11, i_brand_id#6 AS brand_id#16, i_brand#7 AS brand#17, Mak (17) TakeOrderedAndProject Input [4]: [d_year#11, brand_id#16, brand#17, sum_agg#18] -Arguments: 100, 0, [d_year#11 ASC NULLS FIRST, sum_agg#18 DESC NULLS LAST, brand_id#16 ASC NULLS FIRST], [d_year#11, brand_id#16, brand#17, sum_agg#18] +Arguments: 100, [d_year#11 ASC NULLS FIRST, sum_agg#18 DESC NULLS LAST, brand_id#16 ASC NULLS FIRST], [d_year#11, brand_id#16, brand#17, sum_agg#18] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q3/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q3/explain.txt index afce3e4dad180..a60caf19f23df 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q3/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q3/explain.txt @@ -119,5 +119,5 @@ Results [4]: [d_year#2, i_brand_id#9 AS brand_id#17, i_brand#10 AS brand#18, Mak (21) TakeOrderedAndProject Input [4]: [d_year#2, brand_id#17, brand#18, sum_agg#19] -Arguments: 100, 0, [d_year#2 ASC NULLS FIRST, sum_agg#19 DESC NULLS LAST, brand_id#17 ASC NULLS FIRST], [d_year#2, brand_id#17, brand#18, sum_agg#19] +Arguments: 100, [d_year#2 ASC NULLS FIRST, sum_agg#19 DESC NULLS LAST, brand_id#17 ASC NULLS FIRST], [d_year#2, brand_id#17, brand#18, sum_agg#19] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q42.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q42.sf100/explain.txt index 5f117797460ed..5e83e995a3766 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q42.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q42.sf100/explain.txt @@ -119,7 +119,7 @@ Results [4]: [d_year#2, i_category_id#10, i_category#11, MakeDecimal(sum(Unscale (21) TakeOrderedAndProject Input [4]: [d_year#2, i_category_id#10, i_category#11, sum(ss_ext_sales_price)#18] -Arguments: 100, 0, [sum(ss_ext_sales_price)#18 DESC NULLS LAST, d_year#2 ASC NULLS FIRST, i_category_id#10 ASC NULLS FIRST, i_category#11 ASC NULLS FIRST], [d_year#2, i_category_id#10, i_category#11, sum(ss_ext_sales_price)#18] +Arguments: 100, [sum(ss_ext_sales_price)#18 DESC NULLS LAST, d_year#2 ASC NULLS FIRST, i_category_id#10 ASC NULLS FIRST, i_category#11 ASC NULLS FIRST], [d_year#2, i_category_id#10, i_category#11, sum(ss_ext_sales_price)#18] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q42/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q42/explain.txt index 4fa60d609ecb1..255f39414f908 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q42/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q42/explain.txt @@ -119,5 +119,5 @@ Results [4]: [d_year#2, i_category_id#9, i_category#10, MakeDecimal(sum(Unscaled (21) TakeOrderedAndProject Input [4]: [d_year#2, i_category_id#9, i_category#10, sum(ss_ext_sales_price)#17] -Arguments: 100, 0, [sum(ss_ext_sales_price)#17 DESC NULLS LAST, d_year#2 ASC NULLS FIRST, i_category_id#9 ASC NULLS FIRST, i_category#10 ASC NULLS FIRST], [d_year#2, i_category_id#9, i_category#10, sum(ss_ext_sales_price)#17] +Arguments: 100, [sum(ss_ext_sales_price)#17 DESC NULLS LAST, d_year#2 ASC NULLS FIRST, i_category_id#9 ASC NULLS FIRST, i_category#10 ASC NULLS FIRST], [d_year#2, i_category_id#9, i_category#10, sum(ss_ext_sales_price)#17] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q43.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q43.sf100/explain.txt index c9861a6680fdf..ec49a329ecded 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q43.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q43.sf100/explain.txt @@ -119,7 +119,7 @@ Results [9]: [s_store_name#11, s_store_id#10, MakeDecimal(sum(UnscaledValue(CASE (21) TakeOrderedAndProject Input [9]: [s_store_name#11, s_store_id#10, sun_sales#36, mon_sales#37, tue_sales#38, wed_sales#39, thu_sales#40, fri_sales#41, sat_sales#42] -Arguments: 100, 0, [s_store_name#11 ASC NULLS FIRST, s_store_id#10 ASC NULLS FIRST, sun_sales#36 ASC NULLS FIRST, mon_sales#37 ASC NULLS FIRST, tue_sales#38 ASC NULLS FIRST, wed_sales#39 ASC NULLS FIRST, thu_sales#40 ASC NULLS FIRST, fri_sales#41 ASC NULLS FIRST, sat_sales#42 ASC NULLS FIRST], [s_store_name#11, s_store_id#10, sun_sales#36, mon_sales#37, tue_sales#38, wed_sales#39, thu_sales#40, fri_sales#41, sat_sales#42] +Arguments: 100, [s_store_name#11 ASC NULLS FIRST, s_store_id#10 ASC NULLS FIRST, sun_sales#36 ASC NULLS FIRST, mon_sales#37 ASC NULLS FIRST, tue_sales#38 ASC NULLS FIRST, wed_sales#39 ASC NULLS FIRST, thu_sales#40 ASC NULLS FIRST, fri_sales#41 ASC NULLS FIRST, sat_sales#42 ASC NULLS FIRST], [s_store_name#11, s_store_id#10, sun_sales#36, mon_sales#37, tue_sales#38, wed_sales#39, thu_sales#40, fri_sales#41, sat_sales#42] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q43/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q43/explain.txt index 38ff8714c9615..2c15f0ebcbd54 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q43/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q43/explain.txt @@ -119,5 +119,5 @@ Results [9]: [s_store_name#10, s_store_id#9, MakeDecimal(sum(UnscaledValue(CASE (21) TakeOrderedAndProject Input [9]: [s_store_name#10, s_store_id#9, sun_sales#35, mon_sales#36, tue_sales#37, wed_sales#38, thu_sales#39, fri_sales#40, sat_sales#41] -Arguments: 100, 0, [s_store_name#10 ASC NULLS FIRST, s_store_id#9 ASC NULLS FIRST, sun_sales#35 ASC NULLS FIRST, mon_sales#36 ASC NULLS FIRST, tue_sales#37 ASC NULLS FIRST, wed_sales#38 ASC NULLS FIRST, thu_sales#39 ASC NULLS FIRST, fri_sales#40 ASC NULLS FIRST, sat_sales#41 ASC NULLS FIRST], [s_store_name#10, s_store_id#9, sun_sales#35, mon_sales#36, tue_sales#37, wed_sales#38, thu_sales#39, fri_sales#40, sat_sales#41] +Arguments: 100, [s_store_name#10 ASC NULLS FIRST, s_store_id#9 ASC NULLS FIRST, sun_sales#35 ASC NULLS FIRST, mon_sales#36 ASC NULLS FIRST, tue_sales#37 ASC NULLS FIRST, wed_sales#38 ASC NULLS FIRST, thu_sales#39 ASC NULLS FIRST, fri_sales#40 ASC NULLS FIRST, sat_sales#41 ASC NULLS FIRST], [s_store_name#10, s_store_id#9, sun_sales#35, mon_sales#36, tue_sales#37, wed_sales#38, thu_sales#39, fri_sales#40, sat_sales#41] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q46.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q46.sf100/explain.txt index be323349b900d..4ea916a38b26b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q46.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q46.sf100/explain.txt @@ -255,7 +255,7 @@ Input [9]: [ss_ticket_number#5, bought_city#28, amt#29, profit#30, c_current_add (47) TakeOrderedAndProject Input [7]: [c_last_name#35, c_first_name#34, ca_city#39, bought_city#28, ss_ticket_number#5, amt#29, profit#30] -Arguments: 100, 0, [c_last_name#35 ASC NULLS FIRST, c_first_name#34 ASC NULLS FIRST, ca_city#39 ASC NULLS FIRST, bought_city#28 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#35, c_first_name#34, ca_city#39, bought_city#28, ss_ticket_number#5, amt#29, profit#30] +Arguments: 100, [c_last_name#35 ASC NULLS FIRST, c_first_name#34 ASC NULLS FIRST, ca_city#39 ASC NULLS FIRST, bought_city#28 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#35, c_first_name#34, ca_city#39, bought_city#28, ss_ticket_number#5, amt#29, profit#30] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q46/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q46/explain.txt index 22b8206c0fd24..5d36c1d7cd2f5 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q46/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q46/explain.txt @@ -215,7 +215,7 @@ Input [9]: [ss_ticket_number#5, bought_city#28, amt#29, profit#30, c_current_add (39) TakeOrderedAndProject Input [7]: [c_last_name#34, c_first_name#33, ca_city#37, bought_city#28, ss_ticket_number#5, amt#29, profit#30] -Arguments: 100, 0, [c_last_name#34 ASC NULLS FIRST, c_first_name#33 ASC NULLS FIRST, ca_city#37 ASC NULLS FIRST, bought_city#28 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#34, c_first_name#33, ca_city#37, bought_city#28, ss_ticket_number#5, amt#29, profit#30] +Arguments: 100, [c_last_name#34 ASC NULLS FIRST, c_first_name#33 ASC NULLS FIRST, ca_city#37 ASC NULLS FIRST, bought_city#28 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#34, c_first_name#33, ca_city#37, bought_city#28, ss_ticket_number#5, amt#29, profit#30] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q52.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q52.sf100/explain.txt index 90f1eb77ec15f..02f680fc0dd1a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q52.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q52.sf100/explain.txt @@ -119,7 +119,7 @@ Results [4]: [d_year#2, i_brand_id#10 AS brand_id#18, i_brand#11 AS brand#19, Ma (21) TakeOrderedAndProject Input [4]: [d_year#2, brand_id#18, brand#19, ext_price#20] -Arguments: 100, 0, [d_year#2 ASC NULLS FIRST, ext_price#20 DESC NULLS LAST, brand_id#18 ASC NULLS FIRST], [d_year#2, brand_id#18, brand#19, ext_price#20] +Arguments: 100, [d_year#2 ASC NULLS FIRST, ext_price#20 DESC NULLS LAST, brand_id#18 ASC NULLS FIRST], [d_year#2, brand_id#18, brand#19, ext_price#20] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q52/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q52/explain.txt index 77cf242e5b169..8d081d60f9e85 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q52/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q52/explain.txt @@ -119,5 +119,5 @@ Results [4]: [d_year#2, i_brand_id#9 AS brand_id#17, i_brand#10 AS brand#18, Mak (21) TakeOrderedAndProject Input [4]: [d_year#2, brand_id#17, brand#18, ext_price#19] -Arguments: 100, 0, [d_year#2 ASC NULLS FIRST, ext_price#19 DESC NULLS LAST, brand_id#17 ASC NULLS FIRST], [d_year#2, brand_id#17, brand#18, ext_price#19] +Arguments: 100, [d_year#2 ASC NULLS FIRST, ext_price#19 DESC NULLS LAST, brand_id#17 ASC NULLS FIRST], [d_year#2, brand_id#17, brand#18, ext_price#19] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q53.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q53.sf100/explain.txt index 09ca40b47f838..89434b0b06e8f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q53.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q53.sf100/explain.txt @@ -154,7 +154,7 @@ Input [4]: [i_manufact_id#5, sum_sales#24, _w0#25, avg_quarterly_sales#27] (28) TakeOrderedAndProject Input [3]: [i_manufact_id#5, sum_sales#24, avg_quarterly_sales#27] -Arguments: 100, 0, [avg_quarterly_sales#27 ASC NULLS FIRST, sum_sales#24 ASC NULLS FIRST, i_manufact_id#5 ASC NULLS FIRST], [i_manufact_id#5, sum_sales#24, avg_quarterly_sales#27] +Arguments: 100, [avg_quarterly_sales#27 ASC NULLS FIRST, sum_sales#24 ASC NULLS FIRST, i_manufact_id#5 ASC NULLS FIRST], [i_manufact_id#5, sum_sales#24, avg_quarterly_sales#27] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q53/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q53/explain.txt index 849f576980648..4ad6ced134b5a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q53/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q53/explain.txt @@ -154,7 +154,7 @@ Input [4]: [i_manufact_id#5, sum_sales#24, _w0#25, avg_quarterly_sales#27] (28) TakeOrderedAndProject Input [3]: [i_manufact_id#5, sum_sales#24, avg_quarterly_sales#27] -Arguments: 100, 0, [avg_quarterly_sales#27 ASC NULLS FIRST, sum_sales#24 ASC NULLS FIRST, i_manufact_id#5 ASC NULLS FIRST], [i_manufact_id#5, sum_sales#24, avg_quarterly_sales#27] +Arguments: 100, [avg_quarterly_sales#27 ASC NULLS FIRST, sum_sales#24 ASC NULLS FIRST, i_manufact_id#5 ASC NULLS FIRST], [i_manufact_id#5, sum_sales#24, avg_quarterly_sales#27] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q55.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q55.sf100/explain.txt index fb8b7c9004f03..df4d94f40bc16 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q55.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q55.sf100/explain.txt @@ -119,7 +119,7 @@ Results [3]: [i_brand_id#10 AS brand_id#18, i_brand#11 AS brand#19, MakeDecimal( (21) TakeOrderedAndProject Input [3]: [brand_id#18, brand#19, ext_price#20] -Arguments: 100, 0, [ext_price#20 DESC NULLS LAST, brand_id#18 ASC NULLS FIRST], [brand_id#18, brand#19, ext_price#20] +Arguments: 100, [ext_price#20 DESC NULLS LAST, brand_id#18 ASC NULLS FIRST], [brand_id#18, brand#19, ext_price#20] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q55/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q55/explain.txt index b0c67cb339a73..a10fc3256db33 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q55/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q55/explain.txt @@ -119,5 +119,5 @@ Results [3]: [i_brand_id#9 AS brand_id#17, i_brand#10 AS brand#18, MakeDecimal(s (21) TakeOrderedAndProject Input [3]: [brand_id#17, brand#18, ext_price#19] -Arguments: 100, 0, [ext_price#19 DESC NULLS LAST, brand_id#17 ASC NULLS FIRST], [brand_id#17, brand#18, ext_price#19] +Arguments: 100, [ext_price#19 DESC NULLS LAST, brand_id#17 ASC NULLS FIRST], [brand_id#17, brand#18, ext_price#19] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q59.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q59.sf100/explain.txt index ec154a3a89df6..d500ebfb18c8f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q59.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q59.sf100/explain.txt @@ -288,5 +288,5 @@ Input [18]: [s_store_name1#44, d_week_seq1#45, s_store_id1#46, sun_sales1#47, mo (51) TakeOrderedAndProject Input [10]: [s_store_name1#44, s_store_id1#46, d_week_seq1#45, (sun_sales1 / sun_sales2)#82, (mon_sales1 / mon_sales2)#83, (tue_sales1 / tue_sales1)#84, (wed_sales1 / wed_sales2)#85, (thu_sales1 / thu_sales2)#86, (fri_sales1 / fri_sales2)#87, (sat_sales1 / sat_sales2)#88] -Arguments: 100, 0, [s_store_name1#44 ASC NULLS FIRST, s_store_id1#46 ASC NULLS FIRST, d_week_seq1#45 ASC NULLS FIRST], [s_store_name1#44, s_store_id1#46, d_week_seq1#45, (sun_sales1 / sun_sales2)#82, (mon_sales1 / mon_sales2)#83, (tue_sales1 / tue_sales1)#84, (wed_sales1 / wed_sales2)#85, (thu_sales1 / thu_sales2)#86, (fri_sales1 / fri_sales2)#87, (sat_sales1 / sat_sales2)#88] +Arguments: 100, [s_store_name1#44 ASC NULLS FIRST, s_store_id1#46 ASC NULLS FIRST, d_week_seq1#45 ASC NULLS FIRST], [s_store_name1#44, s_store_id1#46, d_week_seq1#45, (sun_sales1 / sun_sales2)#82, (mon_sales1 / mon_sales2)#83, (tue_sales1 / tue_sales1)#84, (wed_sales1 / wed_sales2)#85, (thu_sales1 / thu_sales2)#86, (fri_sales1 / fri_sales2)#87, (sat_sales1 / sat_sales2)#88] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q59/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q59/explain.txt index ec154a3a89df6..d500ebfb18c8f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q59/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q59/explain.txt @@ -288,5 +288,5 @@ Input [18]: [s_store_name1#44, d_week_seq1#45, s_store_id1#46, sun_sales1#47, mo (51) TakeOrderedAndProject Input [10]: [s_store_name1#44, s_store_id1#46, d_week_seq1#45, (sun_sales1 / sun_sales2)#82, (mon_sales1 / mon_sales2)#83, (tue_sales1 / tue_sales1)#84, (wed_sales1 / wed_sales2)#85, (thu_sales1 / thu_sales2)#86, (fri_sales1 / fri_sales2)#87, (sat_sales1 / sat_sales2)#88] -Arguments: 100, 0, [s_store_name1#44 ASC NULLS FIRST, s_store_id1#46 ASC NULLS FIRST, d_week_seq1#45 ASC NULLS FIRST], [s_store_name1#44, s_store_id1#46, d_week_seq1#45, (sun_sales1 / sun_sales2)#82, (mon_sales1 / mon_sales2)#83, (tue_sales1 / tue_sales1)#84, (wed_sales1 / wed_sales2)#85, (thu_sales1 / thu_sales2)#86, (fri_sales1 / fri_sales2)#87, (sat_sales1 / sat_sales2)#88] +Arguments: 100, [s_store_name1#44 ASC NULLS FIRST, s_store_id1#46 ASC NULLS FIRST, d_week_seq1#45 ASC NULLS FIRST], [s_store_name1#44, s_store_id1#46, d_week_seq1#45, (sun_sales1 / sun_sales2)#82, (mon_sales1 / mon_sales2)#83, (tue_sales1 / tue_sales1)#84, (wed_sales1 / wed_sales2)#85, (thu_sales1 / thu_sales2)#86, (fri_sales1 / fri_sales2)#87, (sat_sales1 / sat_sales2)#88] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q63.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q63.sf100/explain.txt index bda0b173dd4bb..35e087b5f1087 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q63.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q63.sf100/explain.txt @@ -154,7 +154,7 @@ Input [4]: [i_manager_id#5, sum_sales#24, _w0#25, avg_monthly_sales#27] (28) TakeOrderedAndProject Input [3]: [i_manager_id#5, sum_sales#24, avg_monthly_sales#27] -Arguments: 100, 0, [i_manager_id#5 ASC NULLS FIRST, avg_monthly_sales#27 ASC NULLS FIRST, sum_sales#24 ASC NULLS FIRST], [i_manager_id#5, sum_sales#24, avg_monthly_sales#27] +Arguments: 100, [i_manager_id#5 ASC NULLS FIRST, avg_monthly_sales#27 ASC NULLS FIRST, sum_sales#24 ASC NULLS FIRST], [i_manager_id#5, sum_sales#24, avg_monthly_sales#27] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q63/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q63/explain.txt index fa5e6a459951d..94181f55279a8 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q63/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q63/explain.txt @@ -154,7 +154,7 @@ Input [4]: [i_manager_id#5, sum_sales#24, _w0#25, avg_monthly_sales#27] (28) TakeOrderedAndProject Input [3]: [i_manager_id#5, sum_sales#24, avg_monthly_sales#27] -Arguments: 100, 0, [i_manager_id#5 ASC NULLS FIRST, avg_monthly_sales#27 ASC NULLS FIRST, sum_sales#24 ASC NULLS FIRST], [i_manager_id#5, sum_sales#24, avg_monthly_sales#27] +Arguments: 100, [i_manager_id#5 ASC NULLS FIRST, avg_monthly_sales#27 ASC NULLS FIRST, sum_sales#24 ASC NULLS FIRST], [i_manager_id#5, sum_sales#24, avg_monthly_sales#27] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65.sf100/explain.txt index 95d9032416e17..b0db241fd5061 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65.sf100/explain.txt @@ -225,7 +225,7 @@ Input [8]: [ss_item_sk#1, revenue#11, s_store_name#32, i_item_sk#34, i_item_desc (39) TakeOrderedAndProject Input [6]: [s_store_name#32, i_item_desc#35, revenue#11, i_current_price#36, i_wholesale_cost#37, i_brand#38] -Arguments: 100, 0, [s_store_name#32 ASC NULLS FIRST, i_item_desc#35 ASC NULLS FIRST], [s_store_name#32, i_item_desc#35, revenue#11, i_current_price#36, i_wholesale_cost#37, i_brand#38] +Arguments: 100, [s_store_name#32 ASC NULLS FIRST, i_item_desc#35 ASC NULLS FIRST], [s_store_name#32, i_item_desc#35, revenue#11, i_current_price#36, i_wholesale_cost#37, i_brand#38] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65/explain.txt index 1f9a63db2ff89..84dd6c4f3a5d9 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65/explain.txt @@ -225,7 +225,7 @@ Input [9]: [s_store_name#2, ss_store_sk#4, revenue#13, i_item_desc#16, i_current (39) TakeOrderedAndProject Input [6]: [s_store_name#2, i_item_desc#16, revenue#13, i_current_price#17, i_wholesale_cost#18, i_brand#19] -Arguments: 100, 0, [s_store_name#2 ASC NULLS FIRST, i_item_desc#16 ASC NULLS FIRST], [s_store_name#2, i_item_desc#16, revenue#13, i_current_price#17, i_wholesale_cost#18, i_brand#19] +Arguments: 100, [s_store_name#2 ASC NULLS FIRST, i_item_desc#16 ASC NULLS FIRST], [s_store_name#2, i_item_desc#16, revenue#13, i_current_price#17, i_wholesale_cost#18, i_brand#19] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q68.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q68.sf100/explain.txt index c6f480e11efe8..020f7344233b1 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q68.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q68.sf100/explain.txt @@ -248,7 +248,7 @@ Input [10]: [ss_ticket_number#5, bought_city#32, extended_price#33, list_price#3 (45) TakeOrderedAndProject Input [8]: [c_last_name#40, c_first_name#39, ca_city#43, bought_city#32, ss_ticket_number#5, extended_price#33, extended_tax#35, list_price#34] -Arguments: 100, 0, [c_last_name#40 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#40, c_first_name#39, ca_city#43, bought_city#32, ss_ticket_number#5, extended_price#33, extended_tax#35, list_price#34] +Arguments: 100, [c_last_name#40 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#40, c_first_name#39, ca_city#43, bought_city#32, ss_ticket_number#5, extended_price#33, extended_tax#35, list_price#34] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q68/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q68/explain.txt index 2f07a8ae8f6ee..63f007cf1205e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q68/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q68/explain.txt @@ -215,7 +215,7 @@ Input [10]: [ss_ticket_number#5, bought_city#32, extended_price#33, list_price#3 (39) TakeOrderedAndProject Input [8]: [c_last_name#39, c_first_name#38, ca_city#42, bought_city#32, ss_ticket_number#5, extended_price#33, extended_tax#35, list_price#34] -Arguments: 100, 0, [c_last_name#39 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#39, c_first_name#38, ca_city#42, bought_city#32, ss_ticket_number#5, extended_price#33, extended_tax#35, list_price#34] +Arguments: 100, [c_last_name#39 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#39, c_first_name#38, ca_city#42, bought_city#32, ss_ticket_number#5, extended_price#33, extended_tax#35, list_price#34] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q7.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q7.sf100/explain.txt index 9763baa606a58..adbc64c43ff2f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q7.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q7.sf100/explain.txt @@ -167,7 +167,7 @@ Results [5]: [i_item_id#21, avg(ss_quantity#4)#40 AS agg1#44, cast((avg(Unscaled (30) TakeOrderedAndProject Input [5]: [i_item_id#21, agg1#44, agg2#45, agg3#46, agg4#47] -Arguments: 100, 0, [i_item_id#21 ASC NULLS FIRST], [i_item_id#21, agg1#44, agg2#45, agg3#46, agg4#47] +Arguments: 100, [i_item_id#21 ASC NULLS FIRST], [i_item_id#21, agg1#44, agg2#45, agg3#46, agg4#47] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q7/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q7/explain.txt index 3730b1b4d7314..afcfd4816dcad 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q7/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q7/explain.txt @@ -167,7 +167,7 @@ Results [5]: [i_item_id#17, avg(ss_quantity#4)#40 AS agg1#44, cast((avg(Unscaled (30) TakeOrderedAndProject Input [5]: [i_item_id#17, agg1#44, agg2#45, agg3#46, agg4#47] -Arguments: 100, 0, [i_item_id#17 ASC NULLS FIRST], [i_item_id#17, agg1#44, agg2#45, agg3#46, agg4#47] +Arguments: 100, [i_item_id#17 ASC NULLS FIRST], [i_item_id#17, agg1#44, agg2#45, agg3#46, agg4#47] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q79.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q79.sf100/explain.txt index 54d0f801b2a29..739c29798ad6e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q79.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q79.sf100/explain.txt @@ -182,7 +182,7 @@ Input [8]: [ss_ticket_number#5, ss_customer_sk#1, s_city#17, amt#26, profit#27, (33) TakeOrderedAndProject Input [7]: [c_last_name#31, c_first_name#30, substr(s_city, 1, 30)#33, ss_ticket_number#5, amt#26, profit#27, s_city#17] -Arguments: 100, 0, [c_last_name#31 ASC NULLS FIRST, c_first_name#30 ASC NULLS FIRST, substr(s_city#17, 1, 30) ASC NULLS FIRST, profit#27 ASC NULLS FIRST], [c_last_name#31, c_first_name#30, substr(s_city, 1, 30)#33, ss_ticket_number#5, amt#26, profit#27] +Arguments: 100, [c_last_name#31 ASC NULLS FIRST, c_first_name#30 ASC NULLS FIRST, substr(s_city#17, 1, 30) ASC NULLS FIRST, profit#27 ASC NULLS FIRST], [c_last_name#31, c_first_name#30, substr(s_city, 1, 30)#33, ss_ticket_number#5, amt#26, profit#27] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q79/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q79/explain.txt index 22b1ccc37b18d..b00a5500d7c9a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q79/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q79/explain.txt @@ -167,7 +167,7 @@ Input [8]: [ss_ticket_number#5, ss_customer_sk#1, s_city#13, amt#26, profit#27, (30) TakeOrderedAndProject Input [7]: [c_last_name#30, c_first_name#29, substr(s_city, 1, 30)#32, ss_ticket_number#5, amt#26, profit#27, s_city#13] -Arguments: 100, 0, [c_last_name#30 ASC NULLS FIRST, c_first_name#29 ASC NULLS FIRST, substr(s_city#13, 1, 30) ASC NULLS FIRST, profit#27 ASC NULLS FIRST], [c_last_name#30, c_first_name#29, substr(s_city, 1, 30)#32, ss_ticket_number#5, amt#26, profit#27] +Arguments: 100, [c_last_name#30 ASC NULLS FIRST, c_first_name#29 ASC NULLS FIRST, substr(s_city#13, 1, 30) ASC NULLS FIRST, profit#27 ASC NULLS FIRST], [c_last_name#30, c_first_name#29, substr(s_city, 1, 30)#32, ss_ticket_number#5, amt#26, profit#27] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q89.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q89.sf100/explain.txt index 602ba9adfb128..2bee1ed53e5db 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q89.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q89.sf100/explain.txt @@ -149,7 +149,7 @@ Input [9]: [i_category#15, i_class#14, i_brand#13, s_store_name#9, s_company_nam (27) TakeOrderedAndProject Input [8]: [i_category#15, i_class#14, i_brand#13, s_store_name#9, s_company_name#10, d_moy#7, sum_sales#21, avg_monthly_sales#24] -Arguments: 100, 0, [(sum_sales#21 - avg_monthly_sales#24) ASC NULLS FIRST, s_store_name#9 ASC NULLS FIRST], [i_category#15, i_class#14, i_brand#13, s_store_name#9, s_company_name#10, d_moy#7, sum_sales#21, avg_monthly_sales#24] +Arguments: 100, [(sum_sales#21 - avg_monthly_sales#24) ASC NULLS FIRST, s_store_name#9 ASC NULLS FIRST], [i_category#15, i_class#14, i_brand#13, s_store_name#9, s_company_name#10, d_moy#7, sum_sales#21, avg_monthly_sales#24] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q89/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q89/explain.txt index 326eea6affdbf..ca6080d82947b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q89/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q89/explain.txt @@ -149,7 +149,7 @@ Input [9]: [i_category#4, i_class#3, i_brand#2, s_store_name#14, s_company_name# (27) TakeOrderedAndProject Input [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#14, s_company_name#15, d_moy#12, sum_sales#21, avg_monthly_sales#24] -Arguments: 100, 0, [(sum_sales#21 - avg_monthly_sales#24) ASC NULLS FIRST, s_store_name#14 ASC NULLS FIRST], [i_category#4, i_class#3, i_brand#2, s_store_name#14, s_company_name#15, d_moy#12, sum_sales#21, avg_monthly_sales#24] +Arguments: 100, [(sum_sales#21 - avg_monthly_sales#24) ASC NULLS FIRST, s_store_name#14 ASC NULLS FIRST], [i_category#4, i_class#3, i_brand#2, s_store_name#14, s_company_name#15, d_moy#12, sum_sales#21, avg_monthly_sales#24] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.sf100/explain.txt index 7c568cd50bd07..a2735ac0e5047 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.sf100/explain.txt @@ -245,7 +245,7 @@ Input [3]: [ctr_customer_sk#11, c_customer_sk#30, c_customer_id#31] (43) TakeOrderedAndProject Input [1]: [c_customer_id#31] -Arguments: 100, 0, [c_customer_id#31 ASC NULLS FIRST], [c_customer_id#31] +Arguments: 100, [c_customer_id#31 ASC NULLS FIRST], [c_customer_id#31] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/explain.txt index 54ea545fc5935..0f43892ddda6d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/explain.txt @@ -230,7 +230,7 @@ Input [3]: [ctr_customer_sk#11, c_customer_sk#29, c_customer_id#30] (40) TakeOrderedAndProject Input [1]: [c_customer_id#30] -Arguments: 100, 0, [c_customer_id#30 ASC NULLS FIRST], [c_customer_id#30] +Arguments: 100, [c_customer_id#30 ASC NULLS FIRST], [c_customer_id#30] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.sf100/explain.txt index 8e31e1adb1d89..1337d5ed266a3 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.sf100/explain.txt @@ -277,7 +277,7 @@ Results [14]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, count (51) TakeOrderedAndProject Input [14]: [cd_gender#25, cd_marital_status#26, cd_education_status#27, cnt1#38, cd_purchase_estimate#28, cnt2#39, cd_credit_rating#29, cnt3#40, cd_dep_count#30, cnt4#41, cd_dep_employed_count#31, cnt5#42, cd_dep_college_count#32, cnt6#43] -Arguments: 100, 0, [cd_gender#25 ASC NULLS FIRST, cd_marital_status#26 ASC NULLS FIRST, cd_education_status#27 ASC NULLS FIRST, cd_purchase_estimate#28 ASC NULLS FIRST, cd_credit_rating#29 ASC NULLS FIRST, cd_dep_count#30 ASC NULLS FIRST, cd_dep_employed_count#31 ASC NULLS FIRST, cd_dep_college_count#32 ASC NULLS FIRST], [cd_gender#25, cd_marital_status#26, cd_education_status#27, cnt1#38, cd_purchase_estimate#28, cnt2#39, cd_credit_rating#29, cnt3#40, cd_dep_count#30, cnt4#41, cd_dep_employed_count#31, cnt5#42, cd_dep_college_count#32, cnt6#43] +Arguments: 100, [cd_gender#25 ASC NULLS FIRST, cd_marital_status#26 ASC NULLS FIRST, cd_education_status#27 ASC NULLS FIRST, cd_purchase_estimate#28 ASC NULLS FIRST, cd_credit_rating#29 ASC NULLS FIRST, cd_dep_count#30 ASC NULLS FIRST, cd_dep_employed_count#31 ASC NULLS FIRST, cd_dep_college_count#32 ASC NULLS FIRST], [cd_gender#25, cd_marital_status#26, cd_education_status#27, cnt1#38, cd_purchase_estimate#28, cnt2#39, cd_credit_rating#29, cnt3#40, cd_dep_count#30, cnt4#41, cd_dep_employed_count#31, cnt5#42, cd_dep_college_count#32, cnt6#43] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/explain.txt index bde8bf2fa7f0f..80c23b3f113a0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/explain.txt @@ -237,7 +237,7 @@ Results [14]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, count (43) TakeOrderedAndProject Input [14]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cnt1#36, cd_purchase_estimate#26, cnt2#37, cd_credit_rating#27, cnt3#38, cd_dep_count#28, cnt4#39, cd_dep_employed_count#29, cnt5#40, cd_dep_college_count#30, cnt6#41] -Arguments: 100, 0, [cd_gender#23 ASC NULLS FIRST, cd_marital_status#24 ASC NULLS FIRST, cd_education_status#25 ASC NULLS FIRST, cd_purchase_estimate#26 ASC NULLS FIRST, cd_credit_rating#27 ASC NULLS FIRST, cd_dep_count#28 ASC NULLS FIRST, cd_dep_employed_count#29 ASC NULLS FIRST, cd_dep_college_count#30 ASC NULLS FIRST], [cd_gender#23, cd_marital_status#24, cd_education_status#25, cnt1#36, cd_purchase_estimate#26, cnt2#37, cd_credit_rating#27, cnt3#38, cd_dep_count#28, cnt4#39, cd_dep_employed_count#29, cnt5#40, cd_dep_college_count#30, cnt6#41] +Arguments: 100, [cd_gender#23 ASC NULLS FIRST, cd_marital_status#24 ASC NULLS FIRST, cd_education_status#25 ASC NULLS FIRST, cd_purchase_estimate#26 ASC NULLS FIRST, cd_credit_rating#27 ASC NULLS FIRST, cd_dep_count#28 ASC NULLS FIRST, cd_dep_employed_count#29 ASC NULLS FIRST, cd_dep_college_count#30 ASC NULLS FIRST], [cd_gender#23, cd_marital_status#24, cd_education_status#25, cnt1#36, cd_purchase_estimate#26, cnt2#37, cd_credit_rating#27, cnt3#38, cd_dep_count#28, cnt4#39, cd_dep_employed_count#29, cnt5#40, cd_dep_college_count#30, cnt6#41] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.sf100/explain.txt index 9592ea1f23441..d186a969291a7 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.sf100/explain.txt @@ -446,7 +446,7 @@ Input [7]: [customer_id#22, year_total#23, customer_preferred_cust_flag#45, year (81) TakeOrderedAndProject Input [1]: [customer_preferred_cust_flag#45] -Arguments: 100, 0, [customer_preferred_cust_flag#45 ASC NULLS FIRST], [customer_preferred_cust_flag#45] +Arguments: 100, [customer_preferred_cust_flag#45 ASC NULLS FIRST], [customer_preferred_cust_flag#45] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/explain.txt index 2beb7c142116b..0bff392494cd2 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/explain.txt @@ -415,7 +415,7 @@ Input [7]: [customer_id#21, year_total#22, customer_preferred_cust_flag#43, year (73) TakeOrderedAndProject Input [1]: [customer_preferred_cust_flag#43] -Arguments: 100, 0, [customer_preferred_cust_flag#43 ASC NULLS FIRST], [customer_preferred_cust_flag#43] +Arguments: 100, [customer_preferred_cust_flag#43 ASC NULLS FIRST], [customer_preferred_cust_flag#43] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.sf100/explain.txt index b83c1717a52aa..58dc9e1340829 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.sf100/explain.txt @@ -126,7 +126,7 @@ Input [9]: [i_item_desc#8, i_category#11, i_class#10, i_current_price#9, itemrev (23) TakeOrderedAndProject Input [7]: [i_item_desc#8, i_category#11, i_class#10, i_current_price#9, itemrevenue#18, revenueratio#23, i_item_id#7] -Arguments: 100, 0, [i_category#11 ASC NULLS FIRST, i_class#10 ASC NULLS FIRST, i_item_id#7 ASC NULLS FIRST, i_item_desc#8 ASC NULLS FIRST, revenueratio#23 ASC NULLS FIRST], [i_item_desc#8, i_category#11, i_class#10, i_current_price#9, itemrevenue#18, revenueratio#23] +Arguments: 100, [i_category#11 ASC NULLS FIRST, i_class#10 ASC NULLS FIRST, i_item_id#7 ASC NULLS FIRST, i_item_desc#8 ASC NULLS FIRST, revenueratio#23 ASC NULLS FIRST], [i_item_desc#8, i_category#11, i_class#10, i_current_price#9, itemrevenue#18, revenueratio#23] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/explain.txt index 7144c9152e8aa..d10ab7d87cf42 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/explain.txt @@ -111,7 +111,7 @@ Input [9]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemreve (20) TakeOrderedAndProject Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#17, revenueratio#22, i_item_id#6] -Arguments: 100, 0, [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#22 ASC NULLS FIRST], [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#17, revenueratio#22] +Arguments: 100, [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#22 ASC NULLS FIRST], [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#17, revenueratio#22] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/explain.txt index 7aeb27494c408..cb463b13375e2 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/explain.txt @@ -696,7 +696,7 @@ Results [6]: [channel#112, i_brand_id#113, i_class_id#114, i_category_id#115, su (126) TakeOrderedAndProject Input [6]: [channel#112, i_brand_id#113, i_class_id#114, i_category_id#115, sum(sales)#126, sum(number_sales)#127] -Arguments: 100, 0, [channel#112 ASC NULLS FIRST, i_brand_id#113 ASC NULLS FIRST, i_class_id#114 ASC NULLS FIRST, i_category_id#115 ASC NULLS FIRST], [channel#112, i_brand_id#113, i_class_id#114, i_category_id#115, sum(sales)#126, sum(number_sales)#127] +Arguments: 100, [channel#112 ASC NULLS FIRST, i_brand_id#113 ASC NULLS FIRST, i_class_id#114 ASC NULLS FIRST, i_category_id#115 ASC NULLS FIRST], [channel#112, i_brand_id#113, i_class_id#114, i_category_id#115, sum(sales)#126, sum(number_sales)#127] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt index ae467fe5ecfa6..4366f2c5d99d3 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt @@ -601,7 +601,7 @@ Results [6]: [channel#105, i_brand_id#106, i_class_id#107, i_category_id#108, su (107) TakeOrderedAndProject Input [6]: [channel#105, i_brand_id#106, i_class_id#107, i_category_id#108, sum(sales)#119, sum(number_sales)#120] -Arguments: 100, 0, [channel#105 ASC NULLS FIRST, i_brand_id#106 ASC NULLS FIRST, i_class_id#107 ASC NULLS FIRST, i_category_id#108 ASC NULLS FIRST], [channel#105, i_brand_id#106, i_class_id#107, i_category_id#108, sum(sales)#119, sum(number_sales)#120] +Arguments: 100, [channel#105 ASC NULLS FIRST, i_brand_id#106 ASC NULLS FIRST, i_class_id#107 ASC NULLS FIRST, i_category_id#108 ASC NULLS FIRST], [channel#105, i_brand_id#106, i_class_id#107, i_category_id#108, sum(sales)#119, sum(number_sales)#120] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/explain.txt index e41baab617b1b..188a166dbf9c1 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/explain.txt @@ -564,7 +564,7 @@ Join condition: None (102) TakeOrderedAndProject Input [12]: [channel#63, i_brand_id#49, i_class_id#50, i_category_id#51, sales#64, number_sales#65, channel#88, i_brand_id#76, i_class_id#77, i_category_id#78, sales#89, number_sales#90] -Arguments: 100, 0, [i_brand_id#49 ASC NULLS FIRST, i_class_id#50 ASC NULLS FIRST, i_category_id#51 ASC NULLS FIRST], [channel#63, i_brand_id#49, i_class_id#50, i_category_id#51, sales#64, number_sales#65, channel#88, i_brand_id#76, i_class_id#77, i_category_id#78, sales#89, number_sales#90] +Arguments: 100, [i_brand_id#49 ASC NULLS FIRST, i_class_id#50 ASC NULLS FIRST, i_category_id#51 ASC NULLS FIRST], [channel#63, i_brand_id#49, i_class_id#50, i_category_id#51, sales#64, number_sales#65, channel#88, i_brand_id#76, i_class_id#77, i_category_id#78, sales#89, number_sales#90] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt index 1a9a4b4952efa..1890fdd496a94 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt @@ -484,7 +484,7 @@ Join condition: None (86) TakeOrderedAndProject Input [12]: [channel#58, i_brand_id#44, i_class_id#45, i_category_id#46, sales#59, number_sales#60, channel#82, i_brand_id#69, i_class_id#70, i_category_id#71, sales#83, number_sales#84] -Arguments: 100, 0, [i_brand_id#44 ASC NULLS FIRST, i_class_id#45 ASC NULLS FIRST, i_category_id#46 ASC NULLS FIRST], [channel#58, i_brand_id#44, i_class_id#45, i_category_id#46, sales#59, number_sales#60, channel#82, i_brand_id#69, i_class_id#70, i_category_id#71, sales#83, number_sales#84] +Arguments: 100, [i_brand_id#44 ASC NULLS FIRST, i_class_id#45 ASC NULLS FIRST, i_category_id#46 ASC NULLS FIRST], [channel#58, i_brand_id#44, i_class_id#45, i_category_id#46, sales#59, number_sales#60, channel#82, i_brand_id#69, i_class_id#70, i_category_id#71, sales#83, number_sales#84] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.sf100/explain.txt index 61a201b89f16f..55c39ed2e3288 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.sf100/explain.txt @@ -154,7 +154,7 @@ Results [2]: [ca_zip#12, MakeDecimal(sum(UnscaledValue(cs_sales_price#2))#18,17, (28) TakeOrderedAndProject Input [2]: [ca_zip#12, sum(cs_sales_price)#19] -Arguments: 100, 0, [ca_zip#12 ASC NULLS FIRST], [ca_zip#12, sum(cs_sales_price)#19] +Arguments: 100, [ca_zip#12 ASC NULLS FIRST], [ca_zip#12, sum(cs_sales_price)#19] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/explain.txt index 4f45cbc96bc08..fabc11130f172 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/explain.txt @@ -124,7 +124,7 @@ Results [2]: [ca_zip#10, MakeDecimal(sum(UnscaledValue(cs_sales_price#2))#16,17, (22) TakeOrderedAndProject Input [2]: [ca_zip#10, sum(cs_sales_price)#17] -Arguments: 100, 0, [ca_zip#10 ASC NULLS FIRST], [ca_zip#10, sum(cs_sales_price)#17] +Arguments: 100, [ca_zip#10 ASC NULLS FIRST], [ca_zip#10, sum(cs_sales_price)#17] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/explain.txt index db3a89f04672e..d61798f6ad06e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/explain.txt @@ -267,7 +267,7 @@ Results [15]: [i_item_id#14, i_item_desc#15, s_state#10, count(ss_quantity#5)#70 (49) TakeOrderedAndProject Input [15]: [i_item_id#14, i_item_desc#15, s_state#10, store_sales_quantitycount#79, store_sales_quantityave#80, store_sales_quantitystdev#81, store_sales_quantitycov#82, as_store_returns_quantitycount#83, as_store_returns_quantityave#84, as_store_returns_quantitystdev#85, store_returns_quantitycov#86, catalog_sales_quantitycount#87, catalog_sales_quantityave#88, catalog_sales_quantitystdev#89, catalog_sales_quantitycov#90] -Arguments: 100, 0, [i_item_id#14 ASC NULLS FIRST, i_item_desc#15 ASC NULLS FIRST, s_state#10 ASC NULLS FIRST], [i_item_id#14, i_item_desc#15, s_state#10, store_sales_quantitycount#79, store_sales_quantityave#80, store_sales_quantitystdev#81, store_sales_quantitycov#82, as_store_returns_quantitycount#83, as_store_returns_quantityave#84, as_store_returns_quantitystdev#85, store_returns_quantitycov#86, catalog_sales_quantitycount#87, catalog_sales_quantityave#88, catalog_sales_quantitystdev#89, catalog_sales_quantitycov#90] +Arguments: 100, [i_item_id#14 ASC NULLS FIRST, i_item_desc#15 ASC NULLS FIRST, s_state#10 ASC NULLS FIRST], [i_item_id#14, i_item_desc#15, s_state#10, store_sales_quantitycount#79, store_sales_quantityave#80, store_sales_quantitystdev#81, store_sales_quantitycov#82, as_store_returns_quantitycount#83, as_store_returns_quantityave#84, as_store_returns_quantitystdev#85, store_returns_quantitycov#86, catalog_sales_quantitycount#87, catalog_sales_quantityave#88, catalog_sales_quantitystdev#89, catalog_sales_quantitycov#90] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/explain.txt index abafb13f5cb71..d63f34438d5da 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/explain.txt @@ -222,7 +222,7 @@ Results [15]: [i_item_id#27, i_item_desc#28, s_state#24, count(ss_quantity#5)#67 (40) TakeOrderedAndProject Input [15]: [i_item_id#27, i_item_desc#28, s_state#24, store_sales_quantitycount#76, store_sales_quantityave#77, store_sales_quantitystdev#78, store_sales_quantitycov#79, as_store_returns_quantitycount#80, as_store_returns_quantityave#81, as_store_returns_quantitystdev#82, store_returns_quantitycov#83, catalog_sales_quantitycount#84, catalog_sales_quantityave#85, catalog_sales_quantitystdev#86, catalog_sales_quantitycov#87] -Arguments: 100, 0, [i_item_id#27 ASC NULLS FIRST, i_item_desc#28 ASC NULLS FIRST, s_state#24 ASC NULLS FIRST], [i_item_id#27, i_item_desc#28, s_state#24, store_sales_quantitycount#76, store_sales_quantityave#77, store_sales_quantitystdev#78, store_sales_quantitycov#79, as_store_returns_quantitycount#80, as_store_returns_quantityave#81, as_store_returns_quantitystdev#82, store_returns_quantitycov#83, catalog_sales_quantitycount#84, catalog_sales_quantityave#85, catalog_sales_quantitystdev#86, catalog_sales_quantitycov#87] +Arguments: 100, [i_item_id#27 ASC NULLS FIRST, i_item_desc#28 ASC NULLS FIRST, s_state#24 ASC NULLS FIRST], [i_item_id#27, i_item_desc#28, s_state#24, store_sales_quantitycount#76, store_sales_quantityave#77, store_sales_quantitystdev#78, store_sales_quantitycov#79, as_store_returns_quantitycount#80, as_store_returns_quantityave#81, as_store_returns_quantitystdev#82, store_returns_quantitycov#83, catalog_sales_quantitycount#84, catalog_sales_quantityave#85, catalog_sales_quantitystdev#86, catalog_sales_quantitycov#87] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.sf100/explain.txt index a0c7ad80898a7..242ef26397ae2 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.sf100/explain.txt @@ -268,7 +268,7 @@ Results [11]: [i_item_id#35, ca_country#36, ca_state#37, ca_county#38, avg(cast( (49) TakeOrderedAndProject Input [11]: [i_item_id#35, ca_country#36, ca_state#37, ca_county#38, agg1#76, agg2#77, agg3#78, agg4#79, agg5#80, agg6#81, agg7#82] -Arguments: 100, 0, [ca_country#36 ASC NULLS FIRST, ca_state#37 ASC NULLS FIRST, ca_county#38 ASC NULLS FIRST, i_item_id#35 ASC NULLS FIRST], [i_item_id#35, ca_country#36, ca_state#37, ca_county#38, agg1#76, agg2#77, agg3#78, agg4#79, agg5#80, agg6#81, agg7#82] +Arguments: 100, [ca_country#36 ASC NULLS FIRST, ca_state#37 ASC NULLS FIRST, ca_county#38 ASC NULLS FIRST, i_item_id#35 ASC NULLS FIRST], [i_item_id#35, ca_country#36, ca_state#37, ca_county#38, agg1#76, agg2#77, agg3#78, agg4#79, agg5#80, agg6#81, agg7#82] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/explain.txt index 87acb1b8f31a2..d1b5044290a25 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/explain.txt @@ -238,7 +238,7 @@ Results [11]: [i_item_id#33, ca_country#34, ca_state#35, ca_county#36, avg(cast( (43) TakeOrderedAndProject Input [11]: [i_item_id#33, ca_country#34, ca_state#35, ca_county#36, agg1#74, agg2#75, agg3#76, agg4#77, agg5#78, agg6#79, agg7#80] -Arguments: 100, 0, [ca_country#34 ASC NULLS FIRST, ca_state#35 ASC NULLS FIRST, ca_county#36 ASC NULLS FIRST, i_item_id#33 ASC NULLS FIRST], [i_item_id#33, ca_country#34, ca_state#35, ca_county#36, agg1#74, agg2#75, agg3#76, agg4#77, agg5#78, agg6#79, agg7#80] +Arguments: 100, [ca_country#34 ASC NULLS FIRST, ca_state#35 ASC NULLS FIRST, ca_county#36 ASC NULLS FIRST, i_item_id#33 ASC NULLS FIRST], [i_item_id#33, ca_country#34, ca_state#35, ca_county#36, agg1#74, agg2#75, agg3#76, agg4#77, agg5#78, agg6#79, agg7#80] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19.sf100/explain.txt index 1c6127b9c236b..980ca7ba560c9 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19.sf100/explain.txt @@ -225,7 +225,7 @@ Results [5]: [i_brand_id#8 AS brand_id#30, i_brand#9 AS brand#31, i_manufact_id# (41) TakeOrderedAndProject Input [5]: [brand_id#30, brand#31, i_manufact_id#10, i_manufact#11, ext_price#32] -Arguments: 100, 0, [ext_price#32 DESC NULLS LAST, brand#31 ASC NULLS FIRST, brand_id#30 ASC NULLS FIRST, i_manufact_id#10 ASC NULLS FIRST, i_manufact#11 ASC NULLS FIRST], [brand_id#30, brand#31, i_manufact_id#10, i_manufact#11, ext_price#32] +Arguments: 100, [ext_price#32 DESC NULLS LAST, brand#31 ASC NULLS FIRST, brand_id#30 ASC NULLS FIRST, i_manufact_id#10 ASC NULLS FIRST, i_manufact#11 ASC NULLS FIRST], [brand_id#30, brand#31, i_manufact_id#10, i_manufact#11, ext_price#32] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19/explain.txt index 8c2fb5402dfe3..e321ddc156860 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19/explain.txt @@ -218,5 +218,5 @@ Results [5]: [i_brand_id#11 AS brand_id#30, i_brand#12 AS brand#31, i_manufact_i (39) TakeOrderedAndProject Input [5]: [brand_id#30, brand#31, i_manufact_id#13, i_manufact#14, ext_price#32] -Arguments: 100, 0, [ext_price#32 DESC NULLS LAST, brand#31 ASC NULLS FIRST, brand_id#30 ASC NULLS FIRST, i_manufact_id#13 ASC NULLS FIRST, i_manufact#14 ASC NULLS FIRST], [brand_id#30, brand#31, i_manufact_id#13, i_manufact#14, ext_price#32] +Arguments: 100, [ext_price#32 DESC NULLS LAST, brand#31 ASC NULLS FIRST, brand_id#30 ASC NULLS FIRST, i_manufact_id#13 ASC NULLS FIRST, i_manufact#14 ASC NULLS FIRST], [brand_id#30, brand#31, i_manufact_id#13, i_manufact#14, ext_price#32] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.sf100/explain.txt index 6f86939e32f8b..fb32206c5d32d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.sf100/explain.txt @@ -126,7 +126,7 @@ Input [9]: [i_item_desc#8, i_category#11, i_class#10, i_current_price#9, itemrev (23) TakeOrderedAndProject Input [7]: [i_item_desc#8, i_category#11, i_class#10, i_current_price#9, itemrevenue#18, revenueratio#23, i_item_id#7] -Arguments: 100, 0, [i_category#11 ASC NULLS FIRST, i_class#10 ASC NULLS FIRST, i_item_id#7 ASC NULLS FIRST, i_item_desc#8 ASC NULLS FIRST, revenueratio#23 ASC NULLS FIRST], [i_item_desc#8, i_category#11, i_class#10, i_current_price#9, itemrevenue#18, revenueratio#23] +Arguments: 100, [i_category#11 ASC NULLS FIRST, i_class#10 ASC NULLS FIRST, i_item_id#7 ASC NULLS FIRST, i_item_desc#8 ASC NULLS FIRST, revenueratio#23 ASC NULLS FIRST], [i_item_desc#8, i_category#11, i_class#10, i_current_price#9, itemrevenue#18, revenueratio#23] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/explain.txt index 3c8daf552647d..5fb56166d8cef 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/explain.txt @@ -111,7 +111,7 @@ Input [9]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemreve (20) TakeOrderedAndProject Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#17, revenueratio#22, i_item_id#6] -Arguments: 100, 0, [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#22 ASC NULLS FIRST], [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#17, revenueratio#22] +Arguments: 100, [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#22 ASC NULLS FIRST], [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#17, revenueratio#22] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.sf100/explain.txt index df8420493d7dc..96e1bacaa55ae 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.sf100/explain.txt @@ -134,7 +134,7 @@ Condition : ((isnotnull(inv_before#22) AND isnotnull(inv_after#23)) AND (((inv_b (24) TakeOrderedAndProject Input [4]: [w_warehouse_name#13, i_item_id#7, inv_before#22, inv_after#23] -Arguments: 100, 0, [w_warehouse_name#13 ASC NULLS FIRST, i_item_id#7 ASC NULLS FIRST], [w_warehouse_name#13, i_item_id#7, inv_before#22, inv_after#23] +Arguments: 100, [w_warehouse_name#13 ASC NULLS FIRST, i_item_id#7 ASC NULLS FIRST], [w_warehouse_name#13, i_item_id#7, inv_before#22, inv_after#23] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/explain.txt index 318b7dd31666f..01bba5ee5f966 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/explain.txt @@ -134,7 +134,7 @@ Condition : ((isnotnull(inv_before#22) AND isnotnull(inv_after#23)) AND (((inv_b (24) TakeOrderedAndProject Input [4]: [w_warehouse_name#7, i_item_id#10, inv_before#22, inv_after#23] -Arguments: 100, 0, [w_warehouse_name#7 ASC NULLS FIRST, i_item_id#10 ASC NULLS FIRST], [w_warehouse_name#7, i_item_id#10, inv_before#22, inv_after#23] +Arguments: 100, [w_warehouse_name#7 ASC NULLS FIRST, i_item_id#10 ASC NULLS FIRST], [w_warehouse_name#7, i_item_id#10, inv_before#22, inv_after#23] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22.sf100/explain.txt index 6a7908f51c9fb..4258a9c1a71c1 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22.sf100/explain.txt @@ -144,7 +144,7 @@ Results [5]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, avg(inv_ (26) TakeOrderedAndProject Input [5]: [i_product_name#16, i_brand#17, i_class#18, i_category#19, qoh#27] -Arguments: 100, 0, [qoh#27 ASC NULLS FIRST, i_product_name#16 ASC NULLS FIRST, i_brand#17 ASC NULLS FIRST, i_class#18 ASC NULLS FIRST, i_category#19 ASC NULLS FIRST], [i_product_name#16, i_brand#17, i_class#18, i_category#19, qoh#27] +Arguments: 100, [qoh#27 ASC NULLS FIRST, i_product_name#16 ASC NULLS FIRST, i_brand#17 ASC NULLS FIRST, i_class#18 ASC NULLS FIRST, i_category#19 ASC NULLS FIRST], [i_product_name#16, i_brand#17, i_class#18, i_category#19, qoh#27] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22/explain.txt index 5bc98ac492e8b..c1b0b42178114 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22/explain.txt @@ -129,7 +129,7 @@ Results [5]: [i_product_name#15, i_brand#16, i_class#17, i_category#18, avg(inv_ (23) TakeOrderedAndProject Input [5]: [i_product_name#15, i_brand#16, i_class#17, i_category#18, qoh#26] -Arguments: 100, 0, [qoh#26 ASC NULLS FIRST, i_product_name#15 ASC NULLS FIRST, i_brand#16 ASC NULLS FIRST, i_class#17 ASC NULLS FIRST, i_category#18 ASC NULLS FIRST], [i_product_name#15, i_brand#16, i_class#17, i_category#18, qoh#26] +Arguments: 100, [qoh#26 ASC NULLS FIRST, i_product_name#15 ASC NULLS FIRST, i_brand#16 ASC NULLS FIRST, i_class#17 ASC NULLS FIRST, i_category#18 ASC NULLS FIRST], [i_product_name#15, i_brand#16, i_class#17, i_category#18, qoh#26] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.sf100/explain.txt index cb2a4025bf913..d90708aa6d352 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.sf100/explain.txt @@ -712,7 +712,7 @@ Results [3]: [c_last_name#62, c_first_name#61, sum((cast(ws_quantity#53 as decim (129) TakeOrderedAndProject Input [3]: [c_last_name#42, c_first_name#41, sales#50] -Arguments: 100, 0, [c_last_name#42 ASC NULLS FIRST, c_first_name#41 ASC NULLS FIRST, sales#50 ASC NULLS FIRST], [c_last_name#42, c_first_name#41, sales#50] +Arguments: 100, [c_last_name#42 ASC NULLS FIRST, c_first_name#41 ASC NULLS FIRST, sales#50 ASC NULLS FIRST], [c_last_name#42, c_first_name#41, sales#50] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/explain.txt index 671568ae914db..d723aa3a322e3 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/explain.txt @@ -484,7 +484,7 @@ Results [3]: [c_last_name#60, c_first_name#59, sum((cast(ws_quantity#54 as decim (87) TakeOrderedAndProject Input [3]: [c_last_name#41, c_first_name#40, sales#51] -Arguments: 100, 0, [c_last_name#41 ASC NULLS FIRST, c_first_name#40 ASC NULLS FIRST, sales#51 ASC NULLS FIRST], [c_last_name#41, c_first_name#40, sales#51] +Arguments: 100, [c_last_name#41 ASC NULLS FIRST, c_first_name#40 ASC NULLS FIRST, sales#51 ASC NULLS FIRST], [c_last_name#41, c_first_name#40, sales#51] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.sf100/explain.txt index 01661abebc10a..fc55789fab16a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.sf100/explain.txt @@ -267,7 +267,7 @@ Results [7]: [i_item_id#15, i_item_desc#16, s_store_id#10, s_store_name#11, Make (49) TakeOrderedAndProject Input [7]: [i_item_id#15, i_item_desc#16, s_store_id#10, s_store_name#11, store_sales_profit#44, store_returns_loss#45, catalog_sales_profit#46] -Arguments: 100, 0, [i_item_id#15 ASC NULLS FIRST, i_item_desc#16 ASC NULLS FIRST, s_store_id#10 ASC NULLS FIRST, s_store_name#11 ASC NULLS FIRST], [i_item_id#15, i_item_desc#16, s_store_id#10, s_store_name#11, store_sales_profit#44, store_returns_loss#45, catalog_sales_profit#46] +Arguments: 100, [i_item_id#15 ASC NULLS FIRST, i_item_desc#16 ASC NULLS FIRST, s_store_id#10 ASC NULLS FIRST, s_store_name#11 ASC NULLS FIRST], [i_item_id#15, i_item_desc#16, s_store_id#10, s_store_name#11, store_sales_profit#44, store_returns_loss#45, catalog_sales_profit#46] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/explain.txt index 51034f5a10e8a..b9d08511b6dce 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/explain.txt @@ -222,7 +222,7 @@ Results [7]: [i_item_id#28, i_item_desc#29, s_store_id#24, s_store_name#25, Make (40) TakeOrderedAndProject Input [7]: [i_item_id#28, i_item_desc#29, s_store_id#24, s_store_name#25, store_sales_profit#41, store_returns_loss#42, catalog_sales_profit#43] -Arguments: 100, 0, [i_item_id#28 ASC NULLS FIRST, i_item_desc#29 ASC NULLS FIRST, s_store_id#24 ASC NULLS FIRST, s_store_name#25 ASC NULLS FIRST], [i_item_id#28, i_item_desc#29, s_store_id#24, s_store_name#25, store_sales_profit#41, store_returns_loss#42, catalog_sales_profit#43] +Arguments: 100, [i_item_id#28 ASC NULLS FIRST, i_item_desc#29 ASC NULLS FIRST, s_store_id#24 ASC NULLS FIRST, s_store_name#25 ASC NULLS FIRST], [i_item_id#28, i_item_desc#29, s_store_id#24, s_store_name#25, store_sales_profit#41, store_returns_loss#42, catalog_sales_profit#43] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.sf100/explain.txt index 7ab0a5de79d73..2a9a6d00ba3c8 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.sf100/explain.txt @@ -167,7 +167,7 @@ Results [5]: [i_item_id#21, avg(cs_quantity#4)#40 AS agg1#44, cast((avg(Unscaled (30) TakeOrderedAndProject Input [5]: [i_item_id#21, agg1#44, agg2#45, agg3#46, agg4#47] -Arguments: 100, 0, [i_item_id#21 ASC NULLS FIRST], [i_item_id#21, agg1#44, agg2#45, agg3#46, agg4#47] +Arguments: 100, [i_item_id#21 ASC NULLS FIRST], [i_item_id#21, agg1#44, agg2#45, agg3#46, agg4#47] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/explain.txt index c4b3f35bf514e..bde65bfda5cbf 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/explain.txt @@ -167,7 +167,7 @@ Results [5]: [i_item_id#17, avg(cs_quantity#4)#40 AS agg1#44, cast((avg(Unscaled (30) TakeOrderedAndProject Input [5]: [i_item_id#17, agg1#44, agg2#45, agg3#46, agg4#47] -Arguments: 100, 0, [i_item_id#17 ASC NULLS FIRST], [i_item_id#17, agg1#44, agg2#45, agg3#46, agg4#47] +Arguments: 100, [i_item_id#17 ASC NULLS FIRST], [i_item_id#17, agg1#44, agg2#45, agg3#46, agg4#47] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.sf100/explain.txt index 8adc29aba28e4..7600710283f2a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.sf100/explain.txt @@ -167,7 +167,7 @@ Results [7]: [i_item_id#22, s_state#23, cast((shiftright(spark_grouping_id#24, 0 (30) TakeOrderedAndProject Input [7]: [i_item_id#22, s_state#23, g_state#46, agg1#47, agg2#48, agg3#49, agg4#50] -Arguments: 100, 0, [i_item_id#22 ASC NULLS FIRST, s_state#23 ASC NULLS FIRST], [i_item_id#22, s_state#23, g_state#46, agg1#47, agg2#48, agg3#49, agg4#50] +Arguments: 100, [i_item_id#22 ASC NULLS FIRST, s_state#23 ASC NULLS FIRST], [i_item_id#22, s_state#23, g_state#46, agg1#47, agg2#48, agg3#49, agg4#50] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/explain.txt index 8adc29aba28e4..7600710283f2a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/explain.txt @@ -167,7 +167,7 @@ Results [7]: [i_item_id#22, s_state#23, cast((shiftright(spark_grouping_id#24, 0 (30) TakeOrderedAndProject Input [7]: [i_item_id#22, s_state#23, g_state#46, agg1#47, agg2#48, agg3#49, agg4#50] -Arguments: 100, 0, [i_item_id#22 ASC NULLS FIRST, s_state#23 ASC NULLS FIRST], [i_item_id#22, s_state#23, g_state#46, agg1#47, agg2#48, agg3#49, agg4#50] +Arguments: 100, [i_item_id#22 ASC NULLS FIRST, s_state#23 ASC NULLS FIRST], [i_item_id#22, s_state#23, g_state#46, agg1#47, agg2#48, agg3#49, agg4#50] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.sf100/explain.txt index 12e850cf82ce8..221439075d24d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.sf100/explain.txt @@ -267,7 +267,7 @@ Results [7]: [i_item_id#15, i_item_desc#16, s_store_id#10, s_store_name#11, sum( (49) TakeOrderedAndProject Input [7]: [i_item_id#15, i_item_desc#16, s_store_id#10, s_store_name#11, store_sales_quantity#45, store_returns_quantity#46, catalog_sales_quantity#47] -Arguments: 100, 0, [i_item_id#15 ASC NULLS FIRST, i_item_desc#16 ASC NULLS FIRST, s_store_id#10 ASC NULLS FIRST, s_store_name#11 ASC NULLS FIRST], [i_item_id#15, i_item_desc#16, s_store_id#10, s_store_name#11, store_sales_quantity#45, store_returns_quantity#46, catalog_sales_quantity#47] +Arguments: 100, [i_item_id#15 ASC NULLS FIRST, i_item_desc#16 ASC NULLS FIRST, s_store_id#10 ASC NULLS FIRST, s_store_name#11 ASC NULLS FIRST], [i_item_id#15, i_item_desc#16, s_store_id#10, s_store_name#11, store_sales_quantity#45, store_returns_quantity#46, catalog_sales_quantity#47] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29/explain.txt index 3bc469d6511ee..86096ab89650c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29/explain.txt @@ -222,7 +222,7 @@ Results [7]: [i_item_id#29, i_item_desc#30, s_store_id#25, s_store_name#26, sum( (40) TakeOrderedAndProject Input [7]: [i_item_id#29, i_item_desc#30, s_store_id#25, s_store_name#26, store_sales_quantity#42, store_returns_quantity#43, catalog_sales_quantity#44] -Arguments: 100, 0, [i_item_id#29 ASC NULLS FIRST, i_item_desc#30 ASC NULLS FIRST, s_store_id#25 ASC NULLS FIRST, s_store_name#26 ASC NULLS FIRST], [i_item_id#29, i_item_desc#30, s_store_id#25, s_store_name#26, store_sales_quantity#42, store_returns_quantity#43, catalog_sales_quantity#44] +Arguments: 100, [i_item_id#29 ASC NULLS FIRST, i_item_desc#30 ASC NULLS FIRST, s_store_id#25 ASC NULLS FIRST, s_store_name#26 ASC NULLS FIRST], [i_item_id#29, i_item_desc#30, s_store_id#25, s_store_name#26, store_sales_quantity#42, store_returns_quantity#43, catalog_sales_quantity#44] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3.sf100/explain.txt index 4796d84bd0540..8ec42f473ee9e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3.sf100/explain.txt @@ -96,7 +96,7 @@ Results [4]: [d_year#11, i_brand_id#6 AS brand_id#16, i_brand#7 AS brand#17, Mak (17) TakeOrderedAndProject Input [4]: [d_year#11, brand_id#16, brand#17, sum_agg#18] -Arguments: 100, 0, [d_year#11 ASC NULLS FIRST, sum_agg#18 DESC NULLS LAST, brand_id#16 ASC NULLS FIRST], [d_year#11, brand_id#16, brand#17, sum_agg#18] +Arguments: 100, [d_year#11 ASC NULLS FIRST, sum_agg#18 DESC NULLS LAST, brand_id#16 ASC NULLS FIRST], [d_year#11, brand_id#16, brand#17, sum_agg#18] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3/explain.txt index b9a6526961f14..77eb7fefd4d05 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3/explain.txt @@ -119,5 +119,5 @@ Results [4]: [d_year#2, i_brand_id#9 AS brand_id#17, i_brand#10 AS brand#18, Mak (21) TakeOrderedAndProject Input [4]: [d_year#2, brand_id#17, brand#18, sum_agg#19] -Arguments: 100, 0, [d_year#2 ASC NULLS FIRST, sum_agg#19 DESC NULLS LAST, brand_id#17 ASC NULLS FIRST], [d_year#2, brand_id#17, brand#18, sum_agg#19] +Arguments: 100, [d_year#2 ASC NULLS FIRST, sum_agg#19 DESC NULLS LAST, brand_id#17 ASC NULLS FIRST], [d_year#2, brand_id#17, brand#18, sum_agg#19] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.sf100/explain.txt index 2ba30392ddf01..7183a1224830c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.sf100/explain.txt @@ -308,7 +308,7 @@ Input [16]: [c_customer_id#2, c_salutation#4, c_first_name#5, c_last_name#6, c_p (55) TakeOrderedAndProject Input [13]: [c_customer_id#2, c_salutation#4, c_first_name#5, c_last_name#6, c_preferred_cust_flag#7, c_birth_day#8, c_birth_month#9, c_birth_year#10, c_birth_country#11, c_login#12, c_email_address#13, c_last_review_date#14, ctr_total_return#35] -Arguments: 100, 0, [c_customer_id#2 ASC NULLS FIRST, c_salutation#4 ASC NULLS FIRST, c_first_name#5 ASC NULLS FIRST, c_last_name#6 ASC NULLS FIRST, c_preferred_cust_flag#7 ASC NULLS FIRST, c_birth_day#8 ASC NULLS FIRST, c_birth_month#9 ASC NULLS FIRST, c_birth_year#10 ASC NULLS FIRST, c_birth_country#11 ASC NULLS FIRST, c_login#12 ASC NULLS FIRST, c_email_address#13 ASC NULLS FIRST, c_last_review_date#14 ASC NULLS FIRST, ctr_total_return#35 ASC NULLS FIRST], [c_customer_id#2, c_salutation#4, c_first_name#5, c_last_name#6, c_preferred_cust_flag#7, c_birth_day#8, c_birth_month#9, c_birth_year#10, c_birth_country#11, c_login#12, c_email_address#13, c_last_review_date#14, ctr_total_return#35] +Arguments: 100, [c_customer_id#2 ASC NULLS FIRST, c_salutation#4 ASC NULLS FIRST, c_first_name#5 ASC NULLS FIRST, c_last_name#6 ASC NULLS FIRST, c_preferred_cust_flag#7 ASC NULLS FIRST, c_birth_day#8 ASC NULLS FIRST, c_birth_month#9 ASC NULLS FIRST, c_birth_year#10 ASC NULLS FIRST, c_birth_country#11 ASC NULLS FIRST, c_login#12 ASC NULLS FIRST, c_email_address#13 ASC NULLS FIRST, c_last_review_date#14 ASC NULLS FIRST, ctr_total_return#35 ASC NULLS FIRST], [c_customer_id#2, c_salutation#4, c_first_name#5, c_last_name#6, c_preferred_cust_flag#7, c_birth_day#8, c_birth_month#9, c_birth_year#10, c_birth_country#11, c_login#12, c_email_address#13, c_last_review_date#14, ctr_total_return#35] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/explain.txt index 8e3c98d415941..e6f2d600efd4c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/explain.txt @@ -278,7 +278,7 @@ Input [15]: [ctr_total_return#16, c_customer_id#30, c_current_addr_sk#31, c_salu (49) TakeOrderedAndProject Input [13]: [c_customer_id#30, c_salutation#32, c_first_name#33, c_last_name#34, c_preferred_cust_flag#35, c_birth_day#36, c_birth_month#37, c_birth_year#38, c_birth_country#39, c_login#40, c_email_address#41, c_last_review_date#42, ctr_total_return#16] -Arguments: 100, 0, [c_customer_id#30 ASC NULLS FIRST, c_salutation#32 ASC NULLS FIRST, c_first_name#33 ASC NULLS FIRST, c_last_name#34 ASC NULLS FIRST, c_preferred_cust_flag#35 ASC NULLS FIRST, c_birth_day#36 ASC NULLS FIRST, c_birth_month#37 ASC NULLS FIRST, c_birth_year#38 ASC NULLS FIRST, c_birth_country#39 ASC NULLS FIRST, c_login#40 ASC NULLS FIRST, c_email_address#41 ASC NULLS FIRST, c_last_review_date#42 ASC NULLS FIRST, ctr_total_return#16 ASC NULLS FIRST], [c_customer_id#30, c_salutation#32, c_first_name#33, c_last_name#34, c_preferred_cust_flag#35, c_birth_day#36, c_birth_month#37, c_birth_year#38, c_birth_country#39, c_login#40, c_email_address#41, c_last_review_date#42, ctr_total_return#16] +Arguments: 100, [c_customer_id#30 ASC NULLS FIRST, c_salutation#32 ASC NULLS FIRST, c_first_name#33 ASC NULLS FIRST, c_last_name#34 ASC NULLS FIRST, c_preferred_cust_flag#35 ASC NULLS FIRST, c_birth_day#36 ASC NULLS FIRST, c_birth_month#37 ASC NULLS FIRST, c_birth_year#38 ASC NULLS FIRST, c_birth_country#39 ASC NULLS FIRST, c_login#40 ASC NULLS FIRST, c_email_address#41 ASC NULLS FIRST, c_last_review_date#42 ASC NULLS FIRST, ctr_total_return#16 ASC NULLS FIRST], [c_customer_id#30, c_salutation#32, c_first_name#33, c_last_name#34, c_preferred_cust_flag#35, c_birth_day#36, c_birth_month#37, c_birth_year#38, c_birth_country#39, c_login#40, c_email_address#41, c_last_review_date#42, ctr_total_return#16] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.sf100/explain.txt index ea53cb5843b84..06bcfd38eae48 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.sf100/explain.txt @@ -150,7 +150,7 @@ Input [2]: [cs_sold_date_sk#19, d_date_sk#20] (27) CollectLimit Input [1]: [excess discount amount #21] -Arguments: 100, 0 +Arguments: 100 ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/explain.txt index f9e2ee52e005b..c00a83d6bd491 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/explain.txt @@ -150,7 +150,7 @@ Input [2]: [cs_sold_date_sk#3, d_date_sk#20] (27) CollectLimit Input [1]: [excess discount amount #21] -Arguments: 100, 0 +Arguments: 100 ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/explain.txt index 6b14fd1b3824d..b1e601e2093ae 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/explain.txt @@ -354,7 +354,7 @@ Results [2]: [i_manufact_id#8, sum(total_sales#20)#52 AS total_sales#53] (63) TakeOrderedAndProject Input [2]: [i_manufact_id#8, total_sales#53] -Arguments: 100, 0, [total_sales#53 ASC NULLS FIRST], [i_manufact_id#8, total_sales#53] +Arguments: 100, [total_sales#53 ASC NULLS FIRST], [i_manufact_id#8, total_sales#53] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt index b791152cb058c..2dc29dcb3a78e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/explain.txt @@ -354,7 +354,7 @@ Results [2]: [i_manufact_id#11, sum(total_sales#20)#52 AS total_sales#53] (63) TakeOrderedAndProject Input [2]: [i_manufact_id#11, total_sales#53] -Arguments: 100, 0, [total_sales#53 ASC NULLS FIRST], [i_manufact_id#11, total_sales#53] +Arguments: 100, [total_sales#53 ASC NULLS FIRST], [i_manufact_id#11, total_sales#53] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.sf100/explain.txt index eaa48e4451ba1..1a3d3ccb75e06 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.sf100/explain.txt @@ -287,7 +287,7 @@ Results [18]: [ca_state#22, cd_gender#26, cd_marital_status#27, count(1)#59 AS c (53) TakeOrderedAndProject Input [18]: [ca_state#22, cd_gender#26, cd_marital_status#27, cnt1#69, min(cd_dep_count)#70, max(cd_dep_count)#71, avg(cd_dep_count)#72, cd_dep_employed_count#29, cnt2#73, min(cd_dep_employed_count)#74, max(cd_dep_employed_count)#75, avg(cd_dep_employed_count)#76, cd_dep_college_count#30, cnt3#77, min(cd_dep_college_count)#78, max(cd_dep_college_count)#79, avg(cd_dep_college_count)#80, cd_dep_count#28] -Arguments: 100, 0, [ca_state#22 ASC NULLS FIRST, cd_gender#26 ASC NULLS FIRST, cd_marital_status#27 ASC NULLS FIRST, cd_dep_count#28 ASC NULLS FIRST, cd_dep_employed_count#29 ASC NULLS FIRST, cd_dep_college_count#30 ASC NULLS FIRST], [ca_state#22, cd_gender#26, cd_marital_status#27, cnt1#69, min(cd_dep_count)#70, max(cd_dep_count)#71, avg(cd_dep_count)#72, cd_dep_employed_count#29, cnt2#73, min(cd_dep_employed_count)#74, max(cd_dep_employed_count)#75, avg(cd_dep_employed_count)#76, cd_dep_college_count#30, cnt3#77, min(cd_dep_college_count)#78, max(cd_dep_college_count)#79, avg(cd_dep_college_count)#80] +Arguments: 100, [ca_state#22 ASC NULLS FIRST, cd_gender#26 ASC NULLS FIRST, cd_marital_status#27 ASC NULLS FIRST, cd_dep_count#28 ASC NULLS FIRST, cd_dep_employed_count#29 ASC NULLS FIRST, cd_dep_college_count#30 ASC NULLS FIRST], [ca_state#22, cd_gender#26, cd_marital_status#27, cnt1#69, min(cd_dep_count)#70, max(cd_dep_count)#71, avg(cd_dep_count)#72, cd_dep_employed_count#29, cnt2#73, min(cd_dep_employed_count)#74, max(cd_dep_employed_count)#75, avg(cd_dep_employed_count)#76, cd_dep_college_count#30, cnt3#77, min(cd_dep_college_count)#78, max(cd_dep_college_count)#79, avg(cd_dep_college_count)#80] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/explain.txt index 17fcde4197573..e5a9030386262 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/explain.txt @@ -232,7 +232,7 @@ Results [18]: [ca_state#20, cd_gender#23, cd_marital_status#24, count(1)#56 AS c (42) TakeOrderedAndProject Input [18]: [ca_state#20, cd_gender#23, cd_marital_status#24, cnt1#66, min(cd_dep_count)#67, max(cd_dep_count)#68, avg(cd_dep_count)#69, cd_dep_employed_count#26, cnt2#70, min(cd_dep_employed_count)#71, max(cd_dep_employed_count)#72, avg(cd_dep_employed_count)#73, cd_dep_college_count#27, cnt3#74, min(cd_dep_college_count)#75, max(cd_dep_college_count)#76, avg(cd_dep_college_count)#77, cd_dep_count#25] -Arguments: 100, 0, [ca_state#20 ASC NULLS FIRST, cd_gender#23 ASC NULLS FIRST, cd_marital_status#24 ASC NULLS FIRST, cd_dep_count#25 ASC NULLS FIRST, cd_dep_employed_count#26 ASC NULLS FIRST, cd_dep_college_count#27 ASC NULLS FIRST], [ca_state#20, cd_gender#23, cd_marital_status#24, cnt1#66, min(cd_dep_count)#67, max(cd_dep_count)#68, avg(cd_dep_count)#69, cd_dep_employed_count#26, cnt2#70, min(cd_dep_employed_count)#71, max(cd_dep_employed_count)#72, avg(cd_dep_employed_count)#73, cd_dep_college_count#27, cnt3#74, min(cd_dep_college_count)#75, max(cd_dep_college_count)#76, avg(cd_dep_college_count)#77] +Arguments: 100, [ca_state#20 ASC NULLS FIRST, cd_gender#23 ASC NULLS FIRST, cd_marital_status#24 ASC NULLS FIRST, cd_dep_count#25 ASC NULLS FIRST, cd_dep_employed_count#26 ASC NULLS FIRST, cd_dep_college_count#27 ASC NULLS FIRST], [ca_state#20, cd_gender#23, cd_marital_status#24, cnt1#66, min(cd_dep_count)#67, max(cd_dep_count)#68, avg(cd_dep_count)#69, cd_dep_employed_count#26, cnt2#70, min(cd_dep_employed_count)#71, max(cd_dep_employed_count)#72, avg(cd_dep_employed_count)#73, cd_dep_college_count#27, cnt3#74, min(cd_dep_college_count)#75, max(cd_dep_college_count)#76, avg(cd_dep_college_count)#77] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.sf100/explain.txt index ee103cbe72375..4f4ba33f89cb1 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.sf100/explain.txt @@ -154,7 +154,7 @@ Input [8]: [gross_margin#25, i_category#15, i_class#16, lochierarchy#26, _w1#27, (28) TakeOrderedAndProject Input [5]: [gross_margin#25, i_category#15, i_class#16, lochierarchy#26, rank_within_parent#31] -Arguments: 100, 0, [lochierarchy#26 DESC NULLS LAST, CASE WHEN (lochierarchy#26 = 0) THEN i_category#15 END ASC NULLS FIRST, rank_within_parent#31 ASC NULLS FIRST], [gross_margin#25, i_category#15, i_class#16, lochierarchy#26, rank_within_parent#31] +Arguments: 100, [lochierarchy#26 DESC NULLS LAST, CASE WHEN (lochierarchy#26 = 0) THEN i_category#15 END ASC NULLS FIRST, rank_within_parent#31 ASC NULLS FIRST], [gross_margin#25, i_category#15, i_class#16, lochierarchy#26, rank_within_parent#31] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/explain.txt index bae0eb2a7a0ae..987b506904a13 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/explain.txt @@ -154,7 +154,7 @@ Input [8]: [gross_margin#25, i_category#15, i_class#16, lochierarchy#26, _w1#27, (28) TakeOrderedAndProject Input [5]: [gross_margin#25, i_category#15, i_class#16, lochierarchy#26, rank_within_parent#31] -Arguments: 100, 0, [lochierarchy#26 DESC NULLS LAST, CASE WHEN (lochierarchy#26 = 0) THEN i_category#15 END ASC NULLS FIRST, rank_within_parent#31 ASC NULLS FIRST], [gross_margin#25, i_category#15, i_class#16, lochierarchy#26, rank_within_parent#31] +Arguments: 100, [lochierarchy#26 DESC NULLS LAST, CASE WHEN (lochierarchy#26 = 0) THEN i_category#15 END ASC NULLS FIRST, rank_within_parent#31 ASC NULLS FIRST], [gross_margin#25, i_category#15, i_class#16, lochierarchy#26, rank_within_parent#31] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.sf100/explain.txt index 0ee0782e60580..df9567ae2c5e9 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.sf100/explain.txt @@ -154,7 +154,7 @@ Results [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] (28) TakeOrderedAndProject Input [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] -Arguments: 100, 0, [i_item_id#2 ASC NULLS FIRST], [i_item_id#2, i_item_desc#3, i_current_price#4] +Arguments: 100, [i_item_id#2 ASC NULLS FIRST], [i_item_id#2, i_item_desc#3, i_current_price#4] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37/explain.txt index cba29c535f245..956b8c98a30c2 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37/explain.txt @@ -139,7 +139,7 @@ Results [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] (25) TakeOrderedAndProject Input [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] -Arguments: 100, 0, [i_item_id#2 ASC NULLS FIRST], [i_item_id#2, i_item_desc#3, i_current_price#4] +Arguments: 100, [i_item_id#2 ASC NULLS FIRST], [i_item_id#2, i_item_desc#3, i_current_price#4] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.sf100/explain.txt index b89e47ac91a85..f65209eecd086 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.sf100/explain.txt @@ -661,7 +661,7 @@ Input [13]: [customer_id#26, customer_id#52, customer_first_name#53, customer_la (120) TakeOrderedAndProject Input [7]: [customer_id#52, customer_first_name#53, customer_last_name#54, customer_preferred_cust_flag#55, customer_birth_country#56, customer_login#57, customer_email_address#58] -Arguments: 100, 0, [customer_id#52 ASC NULLS FIRST, customer_first_name#53 ASC NULLS FIRST, customer_last_name#54 ASC NULLS FIRST, customer_preferred_cust_flag#55 ASC NULLS FIRST, customer_birth_country#56 ASC NULLS FIRST, customer_login#57 ASC NULLS FIRST, customer_email_address#58 ASC NULLS FIRST], [customer_id#52, customer_first_name#53, customer_last_name#54, customer_preferred_cust_flag#55, customer_birth_country#56, customer_login#57, customer_email_address#58] +Arguments: 100, [customer_id#52 ASC NULLS FIRST, customer_first_name#53 ASC NULLS FIRST, customer_last_name#54 ASC NULLS FIRST, customer_preferred_cust_flag#55 ASC NULLS FIRST, customer_birth_country#56 ASC NULLS FIRST, customer_login#57 ASC NULLS FIRST, customer_email_address#58 ASC NULLS FIRST], [customer_id#52, customer_first_name#53, customer_last_name#54, customer_preferred_cust_flag#55, customer_birth_country#56, customer_login#57, customer_email_address#58] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/explain.txt index 4db6fd4b778f8..a4fcf5deb9473 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/explain.txt @@ -626,7 +626,7 @@ Input [13]: [customer_id#25, customer_id#50, customer_first_name#51, customer_la (110) TakeOrderedAndProject Input [7]: [customer_id#50, customer_first_name#51, customer_last_name#52, customer_preferred_cust_flag#53, customer_birth_country#54, customer_login#55, customer_email_address#56] -Arguments: 100, 0, [customer_id#50 ASC NULLS FIRST, customer_first_name#51 ASC NULLS FIRST, customer_last_name#52 ASC NULLS FIRST, customer_preferred_cust_flag#53 ASC NULLS FIRST, customer_birth_country#54 ASC NULLS FIRST, customer_login#55 ASC NULLS FIRST, customer_email_address#56 ASC NULLS FIRST], [customer_id#50, customer_first_name#51, customer_last_name#52, customer_preferred_cust_flag#53, customer_birth_country#54, customer_login#55, customer_email_address#56] +Arguments: 100, [customer_id#50 ASC NULLS FIRST, customer_first_name#51 ASC NULLS FIRST, customer_last_name#52 ASC NULLS FIRST, customer_preferred_cust_flag#53 ASC NULLS FIRST, customer_birth_country#54 ASC NULLS FIRST, customer_login#55 ASC NULLS FIRST, customer_email_address#56 ASC NULLS FIRST], [customer_id#50, customer_first_name#51, customer_last_name#52, customer_preferred_cust_flag#53, customer_birth_country#54, customer_login#55, customer_email_address#56] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40.sf100/explain.txt index a4daf4bc2a6dd..f07851125f451 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40.sf100/explain.txt @@ -182,7 +182,7 @@ Results [4]: [w_state#20, i_item_id#14, sum(CASE WHEN (d_date#18 < 2000-03-11) T (33) TakeOrderedAndProject Input [4]: [w_state#20, i_item_id#14, sales_before#33, sales_after#34] -Arguments: 100, 0, [w_state#20 ASC NULLS FIRST, i_item_id#14 ASC NULLS FIRST], [w_state#20, i_item_id#14, sales_before#33, sales_after#34] +Arguments: 100, [w_state#20 ASC NULLS FIRST, i_item_id#14 ASC NULLS FIRST], [w_state#20, i_item_id#14, sales_before#33, sales_after#34] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/explain.txt index 5e9fbc075e59b..239a44282ea45 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/explain.txt @@ -182,7 +182,7 @@ Results [4]: [w_state#14, i_item_id#17, sum(CASE WHEN (d_date#21 < 2000-03-11) T (33) TakeOrderedAndProject Input [4]: [w_state#14, i_item_id#17, sales_before#33, sales_after#34] -Arguments: 100, 0, [w_state#14 ASC NULLS FIRST, i_item_id#17 ASC NULLS FIRST], [w_state#14, i_item_id#17, sales_before#33, sales_after#34] +Arguments: 100, [w_state#14 ASC NULLS FIRST, i_item_id#17 ASC NULLS FIRST], [w_state#14, i_item_id#17, sales_before#33, sales_after#34] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.sf100/explain.txt index 6b7de9ade3b00..33c03d2b767dd 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.sf100/explain.txt @@ -116,5 +116,5 @@ Results [1]: [i_product_name#3] (20) TakeOrderedAndProject Input [1]: [i_product_name#3] -Arguments: 100, 0, [i_product_name#3 ASC NULLS FIRST], [i_product_name#3] +Arguments: 100, [i_product_name#3 ASC NULLS FIRST], [i_product_name#3] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/explain.txt index 6b7de9ade3b00..33c03d2b767dd 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/explain.txt @@ -116,5 +116,5 @@ Results [1]: [i_product_name#3] (20) TakeOrderedAndProject Input [1]: [i_product_name#3] -Arguments: 100, 0, [i_product_name#3 ASC NULLS FIRST], [i_product_name#3] +Arguments: 100, [i_product_name#3 ASC NULLS FIRST], [i_product_name#3] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42.sf100/explain.txt index c2b578019b9d2..30bd5d90995f4 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42.sf100/explain.txt @@ -96,7 +96,7 @@ Results [4]: [d_year#11, i_category_id#6, i_category#7, MakeDecimal(sum(Unscaled (17) TakeOrderedAndProject Input [4]: [d_year#11, i_category_id#6, i_category#7, sum(ss_ext_sales_price)#16] -Arguments: 100, 0, [sum(ss_ext_sales_price)#16 DESC NULLS LAST, d_year#11 ASC NULLS FIRST, i_category_id#6 ASC NULLS FIRST, i_category#7 ASC NULLS FIRST], [d_year#11, i_category_id#6, i_category#7, sum(ss_ext_sales_price)#16] +Arguments: 100, [sum(ss_ext_sales_price)#16 DESC NULLS LAST, d_year#11 ASC NULLS FIRST, i_category_id#6 ASC NULLS FIRST, i_category#7 ASC NULLS FIRST], [d_year#11, i_category_id#6, i_category#7, sum(ss_ext_sales_price)#16] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42/explain.txt index 964ae1a5ee67f..d16b4bca7c3f4 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42/explain.txt @@ -119,5 +119,5 @@ Results [4]: [d_year#2, i_category_id#9, i_category#10, MakeDecimal(sum(Unscaled (21) TakeOrderedAndProject Input [4]: [d_year#2, i_category_id#9, i_category#10, sum(ss_ext_sales_price)#17] -Arguments: 100, 0, [sum(ss_ext_sales_price)#17 DESC NULLS LAST, d_year#2 ASC NULLS FIRST, i_category_id#9 ASC NULLS FIRST, i_category#10 ASC NULLS FIRST], [d_year#2, i_category_id#9, i_category#10, sum(ss_ext_sales_price)#17] +Arguments: 100, [sum(ss_ext_sales_price)#17 DESC NULLS LAST, d_year#2 ASC NULLS FIRST, i_category_id#9 ASC NULLS FIRST, i_category#10 ASC NULLS FIRST], [d_year#2, i_category_id#9, i_category#10, sum(ss_ext_sales_price)#17] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43.sf100/explain.txt index 109dc09b759a5..d626333970af9 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43.sf100/explain.txt @@ -119,7 +119,7 @@ Results [9]: [s_store_name#11, s_store_id#10, MakeDecimal(sum(UnscaledValue(CASE (21) TakeOrderedAndProject Input [9]: [s_store_name#11, s_store_id#10, sun_sales#36, mon_sales#37, tue_sales#38, wed_sales#39, thu_sales#40, fri_sales#41, sat_sales#42] -Arguments: 100, 0, [s_store_name#11 ASC NULLS FIRST, s_store_id#10 ASC NULLS FIRST, sun_sales#36 ASC NULLS FIRST, mon_sales#37 ASC NULLS FIRST, tue_sales#38 ASC NULLS FIRST, wed_sales#39 ASC NULLS FIRST, thu_sales#40 ASC NULLS FIRST, fri_sales#41 ASC NULLS FIRST, sat_sales#42 ASC NULLS FIRST], [s_store_name#11, s_store_id#10, sun_sales#36, mon_sales#37, tue_sales#38, wed_sales#39, thu_sales#40, fri_sales#41, sat_sales#42] +Arguments: 100, [s_store_name#11 ASC NULLS FIRST, s_store_id#10 ASC NULLS FIRST, sun_sales#36 ASC NULLS FIRST, mon_sales#37 ASC NULLS FIRST, tue_sales#38 ASC NULLS FIRST, wed_sales#39 ASC NULLS FIRST, thu_sales#40 ASC NULLS FIRST, fri_sales#41 ASC NULLS FIRST, sat_sales#42 ASC NULLS FIRST], [s_store_name#11, s_store_id#10, sun_sales#36, mon_sales#37, tue_sales#38, wed_sales#39, thu_sales#40, fri_sales#41, sat_sales#42] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43/explain.txt index 995c7e32cb6b9..24af5d2e813cb 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43/explain.txt @@ -119,5 +119,5 @@ Results [9]: [s_store_name#10, s_store_id#9, MakeDecimal(sum(UnscaledValue(CASE (21) TakeOrderedAndProject Input [9]: [s_store_name#10, s_store_id#9, sun_sales#35, mon_sales#36, tue_sales#37, wed_sales#38, thu_sales#39, fri_sales#40, sat_sales#41] -Arguments: 100, 0, [s_store_name#10 ASC NULLS FIRST, s_store_id#9 ASC NULLS FIRST, sun_sales#35 ASC NULLS FIRST, mon_sales#36 ASC NULLS FIRST, tue_sales#37 ASC NULLS FIRST, wed_sales#38 ASC NULLS FIRST, thu_sales#39 ASC NULLS FIRST, fri_sales#40 ASC NULLS FIRST, sat_sales#41 ASC NULLS FIRST], [s_store_name#10, s_store_id#9, sun_sales#35, mon_sales#36, tue_sales#37, wed_sales#38, thu_sales#39, fri_sales#40, sat_sales#41] +Arguments: 100, [s_store_name#10 ASC NULLS FIRST, s_store_id#9 ASC NULLS FIRST, sun_sales#35 ASC NULLS FIRST, mon_sales#36 ASC NULLS FIRST, tue_sales#37 ASC NULLS FIRST, wed_sales#38 ASC NULLS FIRST, thu_sales#39 ASC NULLS FIRST, fri_sales#40 ASC NULLS FIRST, sat_sales#41 ASC NULLS FIRST], [s_store_name#10, s_store_id#9, sun_sales#35, mon_sales#36, tue_sales#37, wed_sales#38, thu_sales#39, fri_sales#40, sat_sales#41] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.sf100/explain.txt index 2b4e326ed9944..e948e6ab3870f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.sf100/explain.txt @@ -165,7 +165,7 @@ Input [5]: [rnk#16, item_sk#17, i_product_name#22, i_item_sk#24, i_product_name# (31) TakeOrderedAndProject Input [3]: [rnk#16, best_performing#26, worst_performing#27] -Arguments: 100, 0, [rnk#16 ASC NULLS FIRST], [rnk#16, best_performing#26, worst_performing#27] +Arguments: 100, [rnk#16 ASC NULLS FIRST], [rnk#16, best_performing#26, worst_performing#27] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/explain.txt index 69bfc9731853f..fc10739c78c78 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/explain.txt @@ -170,7 +170,7 @@ Input [5]: [rnk#16, item_sk#17, i_product_name#21, i_item_sk#23, i_product_name# (32) TakeOrderedAndProject Input [3]: [rnk#16, best_performing#25, worst_performing#26] -Arguments: 100, 0, [rnk#16 ASC NULLS FIRST], [rnk#16, best_performing#25, worst_performing#26] +Arguments: 100, [rnk#16 ASC NULLS FIRST], [rnk#16, best_performing#25, worst_performing#26] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.sf100/explain.txt index 65644947772d8..d5a234a920211 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.sf100/explain.txt @@ -230,7 +230,7 @@ Results [3]: [ca_zip#17, ca_city#16, MakeDecimal(sum(UnscaledValue(ws_sales_pric (42) TakeOrderedAndProject Input [3]: [ca_zip#17, ca_city#16, sum(ws_sales_price)#27] -Arguments: 100, 0, [ca_zip#17 ASC NULLS FIRST, ca_city#16 ASC NULLS FIRST], [ca_zip#17, ca_city#16, sum(ws_sales_price)#27] +Arguments: 100, [ca_zip#17 ASC NULLS FIRST, ca_city#16 ASC NULLS FIRST], [ca_zip#17, ca_city#16, sum(ws_sales_price)#27] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/explain.txt index 02a95e1d674ff..80f9379309580 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/explain.txt @@ -200,7 +200,7 @@ Results [3]: [ca_zip#12, ca_city#11, MakeDecimal(sum(UnscaledValue(ws_sales_pric (36) TakeOrderedAndProject Input [3]: [ca_zip#12, ca_city#11, sum(ws_sales_price)#25] -Arguments: 100, 0, [ca_zip#12 ASC NULLS FIRST, ca_city#11 ASC NULLS FIRST], [ca_zip#12, ca_city#11, sum(ws_sales_price)#25] +Arguments: 100, [ca_zip#12 ASC NULLS FIRST, ca_city#11 ASC NULLS FIRST], [ca_zip#12, ca_city#11, sum(ws_sales_price)#25] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.sf100/explain.txt index e0bb894b147ee..07017d1b64697 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.sf100/explain.txt @@ -255,7 +255,7 @@ Input [9]: [c_customer_sk#1, c_first_name#3, c_last_name#4, ca_city#7, ss_ticket (47) TakeOrderedAndProject Input [7]: [c_last_name#4, c_first_name#3, ca_city#7, bought_city#36, ss_ticket_number#14, amt#37, profit#38] -Arguments: 100, 0, [c_last_name#4 ASC NULLS FIRST, c_first_name#3 ASC NULLS FIRST, ca_city#7 ASC NULLS FIRST, bought_city#36 ASC NULLS FIRST, ss_ticket_number#14 ASC NULLS FIRST], [c_last_name#4, c_first_name#3, ca_city#7, bought_city#36, ss_ticket_number#14, amt#37, profit#38] +Arguments: 100, [c_last_name#4 ASC NULLS FIRST, c_first_name#3 ASC NULLS FIRST, ca_city#7 ASC NULLS FIRST, bought_city#36 ASC NULLS FIRST, ss_ticket_number#14 ASC NULLS FIRST], [c_last_name#4, c_first_name#3, ca_city#7, bought_city#36, ss_ticket_number#14, amt#37, profit#38] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/explain.txt index 490c675dbcc0b..879f2f2ed51d7 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/explain.txt @@ -215,7 +215,7 @@ Input [9]: [ss_ticket_number#5, bought_city#28, amt#29, profit#30, c_current_add (39) TakeOrderedAndProject Input [7]: [c_last_name#34, c_first_name#33, ca_city#37, bought_city#28, ss_ticket_number#5, amt#29, profit#30] -Arguments: 100, 0, [c_last_name#34 ASC NULLS FIRST, c_first_name#33 ASC NULLS FIRST, ca_city#37 ASC NULLS FIRST, bought_city#28 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#34, c_first_name#33, ca_city#37, bought_city#28, ss_ticket_number#5, amt#29, profit#30] +Arguments: 100, [c_last_name#34 ASC NULLS FIRST, c_first_name#33 ASC NULLS FIRST, ca_city#37 ASC NULLS FIRST, bought_city#28 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#34, c_first_name#33, ca_city#37, bought_city#28, ss_ticket_number#5, amt#29, profit#30] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.sf100/explain.txt index e8776e863f73a..ad62e87b8c217 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.sf100/explain.txt @@ -277,7 +277,7 @@ Input [16]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_ye (52) TakeOrderedAndProject Input [10]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#22, psum#49, nsum#50] -Arguments: 100, 0, [(sum_sales#22 - avg_monthly_sales#26) ASC NULLS FIRST, s_store_name#10 ASC NULLS FIRST], [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#22, psum#49, nsum#50] +Arguments: 100, [(sum_sales#22 - avg_monthly_sales#26) ASC NULLS FIRST, s_store_name#10 ASC NULLS FIRST], [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#22, psum#49, nsum#50] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/explain.txt index 3b92d180a08f7..ad8d4176ea19b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/explain.txt @@ -242,7 +242,7 @@ Input [16]: [i_category#3, i_brand#2, s_store_name#14, s_company_name#15, d_year (45) TakeOrderedAndProject Input [10]: [i_category#3, i_brand#2, s_store_name#14, s_company_name#15, d_year#11, d_moy#12, avg_monthly_sales#25, sum_sales#21, psum#47, nsum#48] -Arguments: 100, 0, [(sum_sales#21 - avg_monthly_sales#25) ASC NULLS FIRST, s_store_name#14 ASC NULLS FIRST], [i_category#3, i_brand#2, s_store_name#14, s_company_name#15, d_year#11, d_moy#12, avg_monthly_sales#25, sum_sales#21, psum#47, nsum#48] +Arguments: 100, [(sum_sales#21 - avg_monthly_sales#25) ASC NULLS FIRST, s_store_name#14 ASC NULLS FIRST], [i_category#3, i_brand#2, s_store_name#14, s_company_name#15, d_year#11, d_moy#12, avg_monthly_sales#25, sum_sales#21, psum#47, nsum#48] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.sf100/explain.txt index 22a678796e2b0..cb5e0c4483205 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.sf100/explain.txt @@ -469,7 +469,7 @@ Results [5]: [channel#39, item#33, return_ratio#34, return_rank#37, currency_ran (86) TakeOrderedAndProject Input [5]: [channel#39, item#33, return_ratio#34, return_rank#37, currency_rank#38] -Arguments: 100, 0, [channel#39 ASC NULLS FIRST, return_rank#37 ASC NULLS FIRST, currency_rank#38 ASC NULLS FIRST], [channel#39, item#33, return_ratio#34, return_rank#37, currency_rank#38] +Arguments: 100, [channel#39 ASC NULLS FIRST, return_rank#37 ASC NULLS FIRST, currency_rank#38 ASC NULLS FIRST], [channel#39, item#33, return_ratio#34, return_rank#37, currency_rank#38] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/explain.txt index 68950e5800e49..be0ce28eeae6b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/explain.txt @@ -424,7 +424,7 @@ Results [5]: [channel#38, item#32, return_ratio#33, return_rank#36, currency_ran (77) TakeOrderedAndProject Input [5]: [channel#38, item#32, return_ratio#33, return_rank#36, currency_rank#37] -Arguments: 100, 0, [channel#38 ASC NULLS FIRST, return_rank#36 ASC NULLS FIRST, currency_rank#37 ASC NULLS FIRST], [channel#38, item#32, return_ratio#33, return_rank#36, currency_rank#37] +Arguments: 100, [channel#38 ASC NULLS FIRST, return_rank#36 ASC NULLS FIRST, currency_rank#37 ASC NULLS FIRST], [channel#38, item#32, return_ratio#33, return_rank#36, currency_rank#37] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/explain.txt index 88baa0df1062e..3d51214fccab0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/explain.txt @@ -429,7 +429,7 @@ Results [5]: [channel#135, id#136, sum(sales#39)#151 AS sales#154, sum(returns#4 (77) TakeOrderedAndProject Input [5]: [channel#135, id#136, sales#154, returns#155, profit#156] -Arguments: 100, 0, [channel#135 ASC NULLS FIRST, id#136 ASC NULLS FIRST], [channel#135, id#136, sales#154, returns#155, profit#156] +Arguments: 100, [channel#135 ASC NULLS FIRST, id#136 ASC NULLS FIRST], [channel#135, id#136, sales#154, returns#155, profit#156] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt index fd07c6e30afe8..41b16be22cf0d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt @@ -414,7 +414,7 @@ Results [5]: [channel#134, id#135, sum(sales#39)#150 AS sales#153, sum(returns#4 (74) TakeOrderedAndProject Input [5]: [channel#134, id#135, sales#153, returns#154, profit#155] -Arguments: 100, 0, [channel#134 ASC NULLS FIRST, id#135 ASC NULLS FIRST], [channel#134, id#135, sales#153, returns#154, profit#155] +Arguments: 100, [channel#134 ASC NULLS FIRST, id#135 ASC NULLS FIRST], [channel#134, id#135, sales#153, returns#154, profit#155] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.sf100/explain.txt index 037eb09915247..97f8d567853e7 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.sf100/explain.txt @@ -173,7 +173,7 @@ Results [15]: [s_store_name#17, s_company_id#18, s_street_number#19, s_street_na (31) TakeOrderedAndProject Input [15]: [s_store_name#17, s_company_id#18, s_street_number#19, s_street_name#20, s_street_type#21, s_suite_number#22, s_city#23, s_county#24, s_state#25, s_zip#26, 30 days #44, 31 - 60 days #45, 61 - 90 days #46, 91 - 120 days #47, >120 days #48] -Arguments: 100, 0, [s_store_name#17 ASC NULLS FIRST, s_company_id#18 ASC NULLS FIRST, s_street_number#19 ASC NULLS FIRST, s_street_name#20 ASC NULLS FIRST, s_street_type#21 ASC NULLS FIRST, s_suite_number#22 ASC NULLS FIRST, s_city#23 ASC NULLS FIRST, s_county#24 ASC NULLS FIRST, s_state#25 ASC NULLS FIRST, s_zip#26 ASC NULLS FIRST], [s_store_name#17, s_company_id#18, s_street_number#19, s_street_name#20, s_street_type#21, s_suite_number#22, s_city#23, s_county#24, s_state#25, s_zip#26, 30 days #44, 31 - 60 days #45, 61 - 90 days #46, 91 - 120 days #47, >120 days #48] +Arguments: 100, [s_store_name#17 ASC NULLS FIRST, s_company_id#18 ASC NULLS FIRST, s_street_number#19 ASC NULLS FIRST, s_street_name#20 ASC NULLS FIRST, s_street_type#21 ASC NULLS FIRST, s_suite_number#22 ASC NULLS FIRST, s_city#23 ASC NULLS FIRST, s_county#24 ASC NULLS FIRST, s_state#25 ASC NULLS FIRST, s_zip#26 ASC NULLS FIRST], [s_store_name#17, s_company_id#18, s_street_number#19, s_street_name#20, s_street_type#21, s_suite_number#22, s_city#23, s_county#24, s_state#25, s_zip#26, 30 days #44, 31 - 60 days #45, 61 - 90 days #46, 91 - 120 days #47, >120 days #48] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50/explain.txt index 881431216c339..060953c80f295 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50/explain.txt @@ -158,7 +158,7 @@ Results [15]: [s_store_name#13, s_company_id#14, s_street_number#15, s_street_na (28) TakeOrderedAndProject Input [15]: [s_store_name#13, s_company_id#14, s_street_number#15, s_street_name#16, s_street_type#17, s_suite_number#18, s_city#19, s_county#20, s_state#21, s_zip#22, 30 days #43, 31 - 60 days #44, 61 - 90 days #45, 91 - 120 days #46, >120 days #47] -Arguments: 100, 0, [s_store_name#13 ASC NULLS FIRST, s_company_id#14 ASC NULLS FIRST, s_street_number#15 ASC NULLS FIRST, s_street_name#16 ASC NULLS FIRST, s_street_type#17 ASC NULLS FIRST, s_suite_number#18 ASC NULLS FIRST, s_city#19 ASC NULLS FIRST, s_county#20 ASC NULLS FIRST, s_state#21 ASC NULLS FIRST, s_zip#22 ASC NULLS FIRST], [s_store_name#13, s_company_id#14, s_street_number#15, s_street_name#16, s_street_type#17, s_suite_number#18, s_city#19, s_county#20, s_state#21, s_zip#22, 30 days #43, 31 - 60 days #44, 61 - 90 days #45, 91 - 120 days #46, >120 days #47] +Arguments: 100, [s_store_name#13 ASC NULLS FIRST, s_company_id#14 ASC NULLS FIRST, s_street_number#15 ASC NULLS FIRST, s_street_name#16 ASC NULLS FIRST, s_street_type#17 ASC NULLS FIRST, s_suite_number#18 ASC NULLS FIRST, s_city#19 ASC NULLS FIRST, s_county#20 ASC NULLS FIRST, s_state#21 ASC NULLS FIRST, s_zip#22 ASC NULLS FIRST], [s_store_name#13, s_company_id#14, s_street_number#15, s_street_name#16, s_street_type#17, s_suite_number#18, s_city#19, s_county#20, s_state#21, s_zip#22, 30 days #43, 31 - 60 days #44, 61 - 90 days #45, 91 - 120 days #46, >120 days #47] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.sf100/explain.txt index 87459c7075f73..51b1ae5837e70 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.sf100/explain.txt @@ -203,7 +203,7 @@ Condition : ((isnotnull(web_cumulative#35) AND isnotnull(store_cumulative#36)) A (37) TakeOrderedAndProject Input [6]: [item_sk#30, d_date#31, web_sales#32, store_sales#33, web_cumulative#35, store_cumulative#36] -Arguments: 100, 0, [item_sk#30 ASC NULLS FIRST, d_date#31 ASC NULLS FIRST], [item_sk#30, d_date#31, web_sales#32, store_sales#33, web_cumulative#35, store_cumulative#36] +Arguments: 100, [item_sk#30 ASC NULLS FIRST, d_date#31 ASC NULLS FIRST], [item_sk#30, d_date#31, web_sales#32, store_sales#33, web_cumulative#35, store_cumulative#36] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/explain.txt index 87459c7075f73..51b1ae5837e70 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/explain.txt @@ -203,7 +203,7 @@ Condition : ((isnotnull(web_cumulative#35) AND isnotnull(store_cumulative#36)) A (37) TakeOrderedAndProject Input [6]: [item_sk#30, d_date#31, web_sales#32, store_sales#33, web_cumulative#35, store_cumulative#36] -Arguments: 100, 0, [item_sk#30 ASC NULLS FIRST, d_date#31 ASC NULLS FIRST], [item_sk#30, d_date#31, web_sales#32, store_sales#33, web_cumulative#35, store_cumulative#36] +Arguments: 100, [item_sk#30 ASC NULLS FIRST, d_date#31 ASC NULLS FIRST], [item_sk#30, d_date#31, web_sales#32, store_sales#33, web_cumulative#35, store_cumulative#36] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52.sf100/explain.txt index c537870505b24..2c8c5e970052f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52.sf100/explain.txt @@ -96,7 +96,7 @@ Results [4]: [d_year#11, i_brand_id#6 AS brand_id#16, i_brand#7 AS brand#17, Mak (17) TakeOrderedAndProject Input [4]: [d_year#11, brand_id#16, brand#17, ext_price#18] -Arguments: 100, 0, [d_year#11 ASC NULLS FIRST, ext_price#18 DESC NULLS LAST, brand_id#16 ASC NULLS FIRST], [d_year#11, brand_id#16, brand#17, ext_price#18] +Arguments: 100, [d_year#11 ASC NULLS FIRST, ext_price#18 DESC NULLS LAST, brand_id#16 ASC NULLS FIRST], [d_year#11, brand_id#16, brand#17, ext_price#18] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52/explain.txt index 7c5ef82c9b2f4..e6e106706fb07 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52/explain.txt @@ -119,5 +119,5 @@ Results [4]: [d_year#2, i_brand_id#9 AS brand_id#17, i_brand#10 AS brand#18, Mak (21) TakeOrderedAndProject Input [4]: [d_year#2, brand_id#17, brand#18, ext_price#19] -Arguments: 100, 0, [d_year#2 ASC NULLS FIRST, ext_price#19 DESC NULLS LAST, brand_id#17 ASC NULLS FIRST], [d_year#2, brand_id#17, brand#18, ext_price#19] +Arguments: 100, [d_year#2 ASC NULLS FIRST, ext_price#19 DESC NULLS LAST, brand_id#17 ASC NULLS FIRST], [d_year#2, brand_id#17, brand#18, ext_price#19] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.sf100/explain.txt index c79f5f4d1020f..475802c40b4ca 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.sf100/explain.txt @@ -154,7 +154,7 @@ Input [4]: [i_manufact_id#5, sum_sales#24, _w0#25, avg_quarterly_sales#27] (28) TakeOrderedAndProject Input [3]: [i_manufact_id#5, sum_sales#24, avg_quarterly_sales#27] -Arguments: 100, 0, [avg_quarterly_sales#27 ASC NULLS FIRST, sum_sales#24 ASC NULLS FIRST, i_manufact_id#5 ASC NULLS FIRST], [i_manufact_id#5, sum_sales#24, avg_quarterly_sales#27] +Arguments: 100, [avg_quarterly_sales#27 ASC NULLS FIRST, sum_sales#24 ASC NULLS FIRST, i_manufact_id#5 ASC NULLS FIRST], [i_manufact_id#5, sum_sales#24, avg_quarterly_sales#27] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/explain.txt index 60c5d8cd45618..0f1b122b830f3 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/explain.txt @@ -154,7 +154,7 @@ Input [4]: [i_manufact_id#5, sum_sales#24, _w0#25, avg_quarterly_sales#27] (28) TakeOrderedAndProject Input [3]: [i_manufact_id#5, sum_sales#24, avg_quarterly_sales#27] -Arguments: 100, 0, [avg_quarterly_sales#27 ASC NULLS FIRST, sum_sales#24 ASC NULLS FIRST, i_manufact_id#5 ASC NULLS FIRST], [i_manufact_id#5, sum_sales#24, avg_quarterly_sales#27] +Arguments: 100, [avg_quarterly_sales#27 ASC NULLS FIRST, sum_sales#24 ASC NULLS FIRST, i_manufact_id#5 ASC NULLS FIRST], [i_manufact_id#5, sum_sales#24, avg_quarterly_sales#27] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.sf100/explain.txt index fe7ebb8dd6fe3..bde0ba3179ca3 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.sf100/explain.txt @@ -332,7 +332,7 @@ Results [3]: [segment#39, count(1)#43 AS num_customers#44, (segment#39 * 50) AS (59) TakeOrderedAndProject Input [3]: [segment#39, num_customers#44, segment_base#45] -Arguments: 100, 0, [segment#39 ASC NULLS FIRST, num_customers#44 ASC NULLS FIRST], [segment#39, num_customers#44, segment_base#45] +Arguments: 100, [segment#39 ASC NULLS FIRST, num_customers#44 ASC NULLS FIRST], [segment#39, num_customers#44, segment_base#45] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/explain.txt index 5d6155e0df030..4e45e0e92f98d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/explain.txt @@ -317,7 +317,7 @@ Results [3]: [segment#40, count(1)#44 AS num_customers#45, (segment#40 * 50) AS (56) TakeOrderedAndProject Input [3]: [segment#40, num_customers#45, segment_base#46] -Arguments: 100, 0, [segment#40 ASC NULLS FIRST, num_customers#45 ASC NULLS FIRST], [segment#40, num_customers#45, segment_base#46] +Arguments: 100, [segment#40 ASC NULLS FIRST, num_customers#45 ASC NULLS FIRST], [segment#40, num_customers#45, segment_base#46] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55.sf100/explain.txt index 3c89db1fd10c7..c6870bd3ac6a3 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55.sf100/explain.txt @@ -96,7 +96,7 @@ Results [3]: [i_brand_id#6 AS brand_id#15, i_brand#7 AS brand#16, MakeDecimal(su (17) TakeOrderedAndProject Input [3]: [brand_id#15, brand#16, ext_price#17] -Arguments: 100, 0, [ext_price#17 DESC NULLS LAST, brand_id#15 ASC NULLS FIRST], [brand_id#15, brand#16, ext_price#17] +Arguments: 100, [ext_price#17 DESC NULLS LAST, brand_id#15 ASC NULLS FIRST], [brand_id#15, brand#16, ext_price#17] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55/explain.txt index 5caf0d26ff2eb..25b423382e332 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55/explain.txt @@ -119,5 +119,5 @@ Results [3]: [i_brand_id#9 AS brand_id#17, i_brand#10 AS brand#18, MakeDecimal(s (21) TakeOrderedAndProject Input [3]: [brand_id#17, brand#18, ext_price#19] -Arguments: 100, 0, [ext_price#19 DESC NULLS LAST, brand_id#17 ASC NULLS FIRST], [brand_id#17, brand#18, ext_price#19] +Arguments: 100, [ext_price#19 DESC NULLS LAST, brand_id#17 ASC NULLS FIRST], [brand_id#17, brand#18, ext_price#19] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/explain.txt index 52e46d8fdb345..a7b59fe5d5ea7 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.sf100/explain.txt @@ -354,7 +354,7 @@ Results [2]: [i_item_id#11, sum(total_sales#20)#52 AS total_sales#53] (63) TakeOrderedAndProject Input [2]: [i_item_id#11, total_sales#53] -Arguments: 100, 0, [total_sales#53 ASC NULLS FIRST], [i_item_id#11, total_sales#53] +Arguments: 100, [total_sales#53 ASC NULLS FIRST], [i_item_id#11, total_sales#53] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/explain.txt index 52e46d8fdb345..a7b59fe5d5ea7 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/explain.txt @@ -354,7 +354,7 @@ Results [2]: [i_item_id#11, sum(total_sales#20)#52 AS total_sales#53] (63) TakeOrderedAndProject Input [2]: [i_item_id#11, total_sales#53] -Arguments: 100, 0, [total_sales#53 ASC NULLS FIRST], [i_item_id#11, total_sales#53] +Arguments: 100, [total_sales#53 ASC NULLS FIRST], [i_item_id#11, total_sales#53] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.sf100/explain.txt index e6d847deb40ca..03f395cd454cb 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.sf100/explain.txt @@ -277,7 +277,7 @@ Input [14]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, sum_sales (52) TakeOrderedAndProject Input [9]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#21, psum#46, nsum#47] -Arguments: 100, 0, [(sum_sales#21 - avg_monthly_sales#25) ASC NULLS FIRST, cc_name#10 ASC NULLS FIRST], [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#21, psum#46, nsum#47] +Arguments: 100, [(sum_sales#21 - avg_monthly_sales#25) ASC NULLS FIRST, cc_name#10 ASC NULLS FIRST], [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#21, psum#46, nsum#47] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/explain.txt index c566579dcfe50..a2df663e63572 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/explain.txt @@ -242,7 +242,7 @@ Input [14]: [i_category#3, i_brand#2, cc_name#14, d_year#11, d_moy#12, sum_sales (45) TakeOrderedAndProject Input [9]: [i_category#3, i_brand#2, cc_name#14, d_year#11, d_moy#12, avg_monthly_sales#24, sum_sales#20, psum#44, nsum#45] -Arguments: 100, 0, [(sum_sales#20 - avg_monthly_sales#24) ASC NULLS FIRST, cc_name#14 ASC NULLS FIRST], [i_category#3, i_brand#2, cc_name#14, d_year#11, d_moy#12, avg_monthly_sales#24, sum_sales#20, psum#44, nsum#45] +Arguments: 100, [(sum_sales#20 - avg_monthly_sales#24) ASC NULLS FIRST, cc_name#14 ASC NULLS FIRST], [i_category#3, i_brand#2, cc_name#14, d_year#11, d_moy#12, avg_monthly_sales#24, sum_sales#20, psum#44, nsum#45] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/explain.txt index 1ddd7c7bc6679..b573b7a6cc39b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/explain.txt @@ -276,7 +276,7 @@ Input [5]: [item_id#13, ss_item_rev#14, cs_item_rev#26, item_id#38, ws_item_rev# (49) TakeOrderedAndProject Input [8]: [item_id#13, ss_item_rev#14, ss_dev#41, cs_item_rev#26, cs_dev#42, ws_item_rev#39, ws_dev#43, average#44] -Arguments: 100, 0, [item_id#13 ASC NULLS FIRST, ss_item_rev#14 ASC NULLS FIRST], [item_id#13, ss_item_rev#14, ss_dev#41, cs_item_rev#26, cs_dev#42, ws_item_rev#39, ws_dev#43, average#44] +Arguments: 100, [item_id#13 ASC NULLS FIRST, ss_item_rev#14 ASC NULLS FIRST], [item_id#13, ss_item_rev#14, ss_dev#41, cs_item_rev#26, cs_dev#42, ws_item_rev#39, ws_dev#43, average#44] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt index fc696498b5259..4178025b05292 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt @@ -276,7 +276,7 @@ Input [5]: [item_id#13, ss_item_rev#14, cs_item_rev#26, item_id#38, ws_item_rev# (49) TakeOrderedAndProject Input [8]: [item_id#13, ss_item_rev#14, ss_dev#41, cs_item_rev#26, cs_dev#42, ws_item_rev#39, ws_dev#43, average#44] -Arguments: 100, 0, [item_id#13 ASC NULLS FIRST, ss_item_rev#14 ASC NULLS FIRST], [item_id#13, ss_item_rev#14, ss_dev#41, cs_item_rev#26, cs_dev#42, ws_item_rev#39, ws_dev#43, average#44] +Arguments: 100, [item_id#13 ASC NULLS FIRST, ss_item_rev#14 ASC NULLS FIRST], [item_id#13, ss_item_rev#14, ss_dev#41, cs_item_rev#26, cs_dev#42, ws_item_rev#39, ws_dev#43, average#44] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59.sf100/explain.txt index a433d1c6d0da0..a91072c13c539 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59.sf100/explain.txt @@ -246,5 +246,5 @@ Input [19]: [s_store_name1#44, d_week_seq1#45, s_store_id1#46, sun_sales1#47, mo (44) TakeOrderedAndProject Input [10]: [s_store_name1#44, s_store_id1#46, d_week_seq1#45, (sun_sales1 / sun_sales2)#77, (mon_sales1 / mon_sales2)#78, (tue_sales1 / tue_sales2)#79, (wed_sales1 / wed_sales2)#80, (thu_sales1 / thu_sales2)#81, (fri_sales1 / fri_sales2)#82, (sat_sales1 / sat_sales2)#83] -Arguments: 100, 0, [s_store_name1#44 ASC NULLS FIRST, s_store_id1#46 ASC NULLS FIRST, d_week_seq1#45 ASC NULLS FIRST], [s_store_name1#44, s_store_id1#46, d_week_seq1#45, (sun_sales1 / sun_sales2)#77, (mon_sales1 / mon_sales2)#78, (tue_sales1 / tue_sales2)#79, (wed_sales1 / wed_sales2)#80, (thu_sales1 / thu_sales2)#81, (fri_sales1 / fri_sales2)#82, (sat_sales1 / sat_sales2)#83] +Arguments: 100, [s_store_name1#44 ASC NULLS FIRST, s_store_id1#46 ASC NULLS FIRST, d_week_seq1#45 ASC NULLS FIRST], [s_store_name1#44, s_store_id1#46, d_week_seq1#45, (sun_sales1 / sun_sales2)#77, (mon_sales1 / mon_sales2)#78, (tue_sales1 / tue_sales2)#79, (wed_sales1 / wed_sales2)#80, (thu_sales1 / thu_sales2)#81, (fri_sales1 / fri_sales2)#82, (sat_sales1 / sat_sales2)#83] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/explain.txt index a433d1c6d0da0..a91072c13c539 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/explain.txt @@ -246,5 +246,5 @@ Input [19]: [s_store_name1#44, d_week_seq1#45, s_store_id1#46, sun_sales1#47, mo (44) TakeOrderedAndProject Input [10]: [s_store_name1#44, s_store_id1#46, d_week_seq1#45, (sun_sales1 / sun_sales2)#77, (mon_sales1 / mon_sales2)#78, (tue_sales1 / tue_sales2)#79, (wed_sales1 / wed_sales2)#80, (thu_sales1 / thu_sales2)#81, (fri_sales1 / fri_sales2)#82, (sat_sales1 / sat_sales2)#83] -Arguments: 100, 0, [s_store_name1#44 ASC NULLS FIRST, s_store_id1#46 ASC NULLS FIRST, d_week_seq1#45 ASC NULLS FIRST], [s_store_name1#44, s_store_id1#46, d_week_seq1#45, (sun_sales1 / sun_sales2)#77, (mon_sales1 / mon_sales2)#78, (tue_sales1 / tue_sales2)#79, (wed_sales1 / wed_sales2)#80, (thu_sales1 / thu_sales2)#81, (fri_sales1 / fri_sales2)#82, (sat_sales1 / sat_sales2)#83] +Arguments: 100, [s_store_name1#44 ASC NULLS FIRST, s_store_id1#46 ASC NULLS FIRST, d_week_seq1#45 ASC NULLS FIRST], [s_store_name1#44, s_store_id1#46, d_week_seq1#45, (sun_sales1 / sun_sales2)#77, (mon_sales1 / mon_sales2)#78, (tue_sales1 / tue_sales2)#79, (wed_sales1 / wed_sales2)#80, (thu_sales1 / thu_sales2)#81, (fri_sales1 / fri_sales2)#82, (sat_sales1 / sat_sales2)#83] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/explain.txt index 45174940ead37..5ba9cf5ab20b6 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.sf100/explain.txt @@ -354,7 +354,7 @@ Results [2]: [i_item_id#11, sum(total_sales#20)#52 AS total_sales#53] (63) TakeOrderedAndProject Input [2]: [i_item_id#11, total_sales#53] -Arguments: 100, 0, [i_item_id#11 ASC NULLS FIRST, total_sales#53 ASC NULLS FIRST], [i_item_id#11, total_sales#53] +Arguments: 100, [i_item_id#11 ASC NULLS FIRST, total_sales#53 ASC NULLS FIRST], [i_item_id#11, total_sales#53] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/explain.txt index 45174940ead37..5ba9cf5ab20b6 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/explain.txt @@ -354,7 +354,7 @@ Results [2]: [i_item_id#11, sum(total_sales#20)#52 AS total_sales#53] (63) TakeOrderedAndProject Input [2]: [i_item_id#11, total_sales#53] -Arguments: 100, 0, [i_item_id#11 ASC NULLS FIRST, total_sales#53 ASC NULLS FIRST], [i_item_id#11, total_sales#53] +Arguments: 100, [i_item_id#11 ASC NULLS FIRST, total_sales#53 ASC NULLS FIRST], [i_item_id#11, total_sales#53] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62.sf100/explain.txt index f43d895b0b0ad..0c82f6182c240 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62.sf100/explain.txt @@ -179,5 +179,5 @@ Results [8]: [_groupingexpression#18 AS substr(w_warehouse_name, 1, 20)#35, sm_t (32) TakeOrderedAndProject Input [8]: [substr(w_warehouse_name, 1, 20)#35, sm_type#13, web_name#10, 30 days #36, 31 - 60 days #37, 61 - 90 days #38, 91 - 120 days #39, >120 days #40] -Arguments: 100, 0, [substr(w_warehouse_name, 1, 20)#35 ASC NULLS FIRST, sm_type#13 ASC NULLS FIRST, web_name#10 ASC NULLS FIRST], [substr(w_warehouse_name, 1, 20)#35, sm_type#13, web_name#10, 30 days #36, 31 - 60 days #37, 61 - 90 days #38, 91 - 120 days #39, >120 days #40] +Arguments: 100, [substr(w_warehouse_name, 1, 20)#35 ASC NULLS FIRST, sm_type#13 ASC NULLS FIRST, web_name#10 ASC NULLS FIRST], [substr(w_warehouse_name, 1, 20)#35, sm_type#13, web_name#10, 30 days #36, 31 - 60 days #37, 61 - 90 days #38, 91 - 120 days #39, >120 days #40] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62/explain.txt index 7c9e44012153e..752025ebea0a5 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62/explain.txt @@ -179,5 +179,5 @@ Results [8]: [_groupingexpression#18 AS substr(w_warehouse_name, 1, 20)#35, sm_t (32) TakeOrderedAndProject Input [8]: [substr(w_warehouse_name, 1, 20)#35, sm_type#10, web_name#13, 30 days #36, 31 - 60 days #37, 61 - 90 days #38, 91 - 120 days #39, >120 days #40] -Arguments: 100, 0, [substr(w_warehouse_name, 1, 20)#35 ASC NULLS FIRST, sm_type#10 ASC NULLS FIRST, web_name#13 ASC NULLS FIRST], [substr(w_warehouse_name, 1, 20)#35, sm_type#10, web_name#13, 30 days #36, 31 - 60 days #37, 61 - 90 days #38, 91 - 120 days #39, >120 days #40] +Arguments: 100, [substr(w_warehouse_name, 1, 20)#35 ASC NULLS FIRST, sm_type#10 ASC NULLS FIRST, web_name#13 ASC NULLS FIRST], [substr(w_warehouse_name, 1, 20)#35, sm_type#10, web_name#13, 30 days #36, 31 - 60 days #37, 61 - 90 days #38, 91 - 120 days #39, >120 days #40] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.sf100/explain.txt index 54cd4f6a8b4e5..eb3e3baf14a3a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.sf100/explain.txt @@ -154,7 +154,7 @@ Input [4]: [i_manager_id#5, sum_sales#24, _w0#25, avg_monthly_sales#27] (28) TakeOrderedAndProject Input [3]: [i_manager_id#5, sum_sales#24, avg_monthly_sales#27] -Arguments: 100, 0, [i_manager_id#5 ASC NULLS FIRST, avg_monthly_sales#27 ASC NULLS FIRST, sum_sales#24 ASC NULLS FIRST], [i_manager_id#5, sum_sales#24, avg_monthly_sales#27] +Arguments: 100, [i_manager_id#5 ASC NULLS FIRST, avg_monthly_sales#27 ASC NULLS FIRST, sum_sales#24 ASC NULLS FIRST], [i_manager_id#5, sum_sales#24, avg_monthly_sales#27] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/explain.txt index ca16f6b732e74..a94546e0a7158 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/explain.txt @@ -154,7 +154,7 @@ Input [4]: [i_manager_id#5, sum_sales#24, _w0#25, avg_monthly_sales#27] (28) TakeOrderedAndProject Input [3]: [i_manager_id#5, sum_sales#24, avg_monthly_sales#27] -Arguments: 100, 0, [i_manager_id#5 ASC NULLS FIRST, avg_monthly_sales#27 ASC NULLS FIRST, sum_sales#24 ASC NULLS FIRST], [i_manager_id#5, sum_sales#24, avg_monthly_sales#27] +Arguments: 100, [i_manager_id#5 ASC NULLS FIRST, avg_monthly_sales#27 ASC NULLS FIRST, sum_sales#24 ASC NULLS FIRST], [i_manager_id#5, sum_sales#24, avg_monthly_sales#27] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.sf100/explain.txt index 51f213a961ca0..50016cc6b5e5a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.sf100/explain.txt @@ -240,7 +240,7 @@ Input [8]: [ss_item_sk#1, revenue#11, s_store_name#31, i_item_sk#34, i_item_desc (42) TakeOrderedAndProject Input [6]: [s_store_name#31, i_item_desc#35, revenue#11, i_current_price#36, i_wholesale_cost#37, i_brand#38] -Arguments: 100, 0, [s_store_name#31 ASC NULLS FIRST, i_item_desc#35 ASC NULLS FIRST], [s_store_name#31, i_item_desc#35, revenue#11, i_current_price#36, i_wholesale_cost#37, i_brand#38] +Arguments: 100, [s_store_name#31 ASC NULLS FIRST, i_item_desc#35 ASC NULLS FIRST], [s_store_name#31, i_item_desc#35, revenue#11, i_current_price#36, i_wholesale_cost#37, i_brand#38] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/explain.txt index 38166fcd0970b..458daedc8f2ca 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/explain.txt @@ -225,7 +225,7 @@ Input [9]: [s_store_name#2, ss_store_sk#4, revenue#13, i_item_desc#16, i_current (39) TakeOrderedAndProject Input [6]: [s_store_name#2, i_item_desc#16, revenue#13, i_current_price#17, i_wholesale_cost#18, i_brand#19] -Arguments: 100, 0, [s_store_name#2 ASC NULLS FIRST, i_item_desc#16 ASC NULLS FIRST], [s_store_name#2, i_item_desc#16, revenue#13, i_current_price#17, i_wholesale_cost#18, i_brand#19] +Arguments: 100, [s_store_name#2 ASC NULLS FIRST, i_item_desc#16 ASC NULLS FIRST], [s_store_name#2, i_item_desc#16, revenue#13, i_current_price#17, i_wholesale_cost#18, i_brand#19] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.sf100/explain.txt index 4e2b84fbd5f59..f26c4cb712bc4 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.sf100/explain.txt @@ -290,7 +290,7 @@ Results [44]: [w_warehouse_name#19, w_warehouse_sq_ft#20, w_city#21, w_county#22 (52) TakeOrderedAndProject Input [44]: [w_warehouse_name#19, w_warehouse_sq_ft#20, w_city#21, w_county#22, w_state#23, w_country#24, ship_carriers#147, year#148, jan_sales#520, feb_sales#521, mar_sales#522, apr_sales#523, may_sales#524, jun_sales#525, jul_sales#526, aug_sales#527, sep_sales#528, oct_sales#529, nov_sales#530, dec_sales#531, jan_sales_per_sq_foot#532, feb_sales_per_sq_foot#533, mar_sales_per_sq_foot#534, apr_sales_per_sq_foot#535, may_sales_per_sq_foot#536, jun_sales_per_sq_foot#537, jul_sales_per_sq_foot#538, aug_sales_per_sq_foot#539, sep_sales_per_sq_foot#540, oct_sales_per_sq_foot#541, nov_sales_per_sq_foot#542, dec_sales_per_sq_foot#543, jan_net#544, feb_net#545, mar_net#546, apr_net#547, may_net#548, jun_net#549, jul_net#550, aug_net#551, sep_net#552, oct_net#553, nov_net#554, dec_net#555] -Arguments: 100, 0, [w_warehouse_name#19 ASC NULLS FIRST], [w_warehouse_name#19, w_warehouse_sq_ft#20, w_city#21, w_county#22, w_state#23, w_country#24, ship_carriers#147, year#148, jan_sales#520, feb_sales#521, mar_sales#522, apr_sales#523, may_sales#524, jun_sales#525, jul_sales#526, aug_sales#527, sep_sales#528, oct_sales#529, nov_sales#530, dec_sales#531, jan_sales_per_sq_foot#532, feb_sales_per_sq_foot#533, mar_sales_per_sq_foot#534, apr_sales_per_sq_foot#535, ... 20 more fields] +Arguments: 100, [w_warehouse_name#19 ASC NULLS FIRST], [w_warehouse_name#19, w_warehouse_sq_ft#20, w_city#21, w_county#22, w_state#23, w_country#24, ship_carriers#147, year#148, jan_sales#520, feb_sales#521, mar_sales#522, apr_sales#523, may_sales#524, jun_sales#525, jul_sales#526, aug_sales#527, sep_sales#528, oct_sales#529, nov_sales#530, dec_sales#531, jan_sales_per_sq_foot#532, feb_sales_per_sq_foot#533, mar_sales_per_sq_foot#534, apr_sales_per_sq_foot#535, ... 20 more fields] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/explain.txt index 75cb3039d13f8..b2f34251c33ca 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/explain.txt @@ -290,7 +290,7 @@ Results [44]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13 (52) TakeOrderedAndProject Input [44]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#147, year#148, jan_sales#520, feb_sales#521, mar_sales#522, apr_sales#523, may_sales#524, jun_sales#525, jul_sales#526, aug_sales#527, sep_sales#528, oct_sales#529, nov_sales#530, dec_sales#531, jan_sales_per_sq_foot#532, feb_sales_per_sq_foot#533, mar_sales_per_sq_foot#534, apr_sales_per_sq_foot#535, may_sales_per_sq_foot#536, jun_sales_per_sq_foot#537, jul_sales_per_sq_foot#538, aug_sales_per_sq_foot#539, sep_sales_per_sq_foot#540, oct_sales_per_sq_foot#541, nov_sales_per_sq_foot#542, dec_sales_per_sq_foot#543, jan_net#544, feb_net#545, mar_net#546, apr_net#547, may_net#548, jun_net#549, jul_net#550, aug_net#551, sep_net#552, oct_net#553, nov_net#554, dec_net#555] -Arguments: 100, 0, [w_warehouse_name#10 ASC NULLS FIRST], [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#147, year#148, jan_sales#520, feb_sales#521, mar_sales#522, apr_sales#523, may_sales#524, jun_sales#525, jul_sales#526, aug_sales#527, sep_sales#528, oct_sales#529, nov_sales#530, dec_sales#531, jan_sales_per_sq_foot#532, feb_sales_per_sq_foot#533, mar_sales_per_sq_foot#534, apr_sales_per_sq_foot#535, ... 20 more fields] +Arguments: 100, [w_warehouse_name#10 ASC NULLS FIRST], [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#147, year#148, jan_sales#520, feb_sales#521, mar_sales#522, apr_sales#523, may_sales#524, jun_sales#525, jul_sales#526, aug_sales#527, sep_sales#528, oct_sales#529, nov_sales#530, dec_sales#531, jan_sales_per_sq_foot#532, feb_sales_per_sq_foot#533, mar_sales_per_sq_foot#534, apr_sales_per_sq_foot#535, ... 20 more fields] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.sf100/explain.txt index 6a2556b1ca700..8c571092e8807 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.sf100/explain.txt @@ -164,7 +164,7 @@ Condition : (rk#38 <= 100) (30) TakeOrderedAndProject Input [10]: [i_category#21, i_class#22, i_brand#23, i_product_name#24, d_year#25, d_qoy#26, d_moy#27, s_store_id#28, sumsales#36, rk#38] -Arguments: 100, 0, [i_category#21 ASC NULLS FIRST, i_class#22 ASC NULLS FIRST, i_brand#23 ASC NULLS FIRST, i_product_name#24 ASC NULLS FIRST, d_year#25 ASC NULLS FIRST, d_qoy#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST, s_store_id#28 ASC NULLS FIRST, sumsales#36 ASC NULLS FIRST, rk#38 ASC NULLS FIRST], [i_category#21, i_class#22, i_brand#23, i_product_name#24, d_year#25, d_qoy#26, d_moy#27, s_store_id#28, sumsales#36, rk#38] +Arguments: 100, [i_category#21 ASC NULLS FIRST, i_class#22 ASC NULLS FIRST, i_brand#23 ASC NULLS FIRST, i_product_name#24 ASC NULLS FIRST, d_year#25 ASC NULLS FIRST, d_qoy#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST, s_store_id#28 ASC NULLS FIRST, sumsales#36 ASC NULLS FIRST, rk#38 ASC NULLS FIRST], [i_category#21, i_class#22, i_brand#23, i_product_name#24, d_year#25, d_qoy#26, d_moy#27, s_store_id#28, sumsales#36, rk#38] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/explain.txt index 2b73a21d0c1ce..83017e0531c4b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/explain.txt @@ -149,7 +149,7 @@ Condition : (rk#37 <= 100) (27) TakeOrderedAndProject Input [10]: [i_category#20, i_class#21, i_brand#22, i_product_name#23, d_year#24, d_qoy#25, d_moy#26, s_store_id#27, sumsales#35, rk#37] -Arguments: 100, 0, [i_category#20 ASC NULLS FIRST, i_class#21 ASC NULLS FIRST, i_brand#22 ASC NULLS FIRST, i_product_name#23 ASC NULLS FIRST, d_year#24 ASC NULLS FIRST, d_qoy#25 ASC NULLS FIRST, d_moy#26 ASC NULLS FIRST, s_store_id#27 ASC NULLS FIRST, sumsales#35 ASC NULLS FIRST, rk#37 ASC NULLS FIRST], [i_category#20, i_class#21, i_brand#22, i_product_name#23, d_year#24, d_qoy#25, d_moy#26, s_store_id#27, sumsales#35, rk#37] +Arguments: 100, [i_category#20 ASC NULLS FIRST, i_class#21 ASC NULLS FIRST, i_brand#22 ASC NULLS FIRST, i_product_name#23 ASC NULLS FIRST, d_year#24 ASC NULLS FIRST, d_qoy#25 ASC NULLS FIRST, d_moy#26 ASC NULLS FIRST, s_store_id#27 ASC NULLS FIRST, sumsales#35 ASC NULLS FIRST, rk#37 ASC NULLS FIRST], [i_category#20, i_class#21, i_brand#22, i_product_name#23, d_year#24, d_qoy#25, d_moy#26, s_store_id#27, sumsales#35, rk#37] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.sf100/explain.txt index ce8cb45f8cc4b..2075dff70a542 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.sf100/explain.txt @@ -255,7 +255,7 @@ Input [10]: [c_customer_sk#1, c_first_name#3, c_last_name#4, ca_city#7, ss_ticke (47) TakeOrderedAndProject Input [8]: [c_last_name#4, c_first_name#3, ca_city#7, bought_city#40, ss_ticket_number#14, extended_price#41, extended_tax#43, list_price#42] -Arguments: 100, 0, [c_last_name#4 ASC NULLS FIRST, ss_ticket_number#14 ASC NULLS FIRST], [c_last_name#4, c_first_name#3, ca_city#7, bought_city#40, ss_ticket_number#14, extended_price#41, extended_tax#43, list_price#42] +Arguments: 100, [c_last_name#4 ASC NULLS FIRST, ss_ticket_number#14 ASC NULLS FIRST], [c_last_name#4, c_first_name#3, ca_city#7, bought_city#40, ss_ticket_number#14, extended_price#41, extended_tax#43, list_price#42] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/explain.txt index bb7750a187793..40336e3e7c909 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/explain.txt @@ -215,7 +215,7 @@ Input [10]: [ss_ticket_number#5, bought_city#32, extended_price#33, list_price#3 (39) TakeOrderedAndProject Input [8]: [c_last_name#39, c_first_name#38, ca_city#42, bought_city#32, ss_ticket_number#5, extended_price#33, extended_tax#35, list_price#34] -Arguments: 100, 0, [c_last_name#39 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#39, c_first_name#38, ca_city#42, bought_city#32, ss_ticket_number#5, extended_price#33, extended_tax#35, list_price#34] +Arguments: 100, [c_last_name#39 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#39, c_first_name#38, ca_city#42, bought_city#32, ss_ticket_number#5, extended_price#33, extended_tax#35, list_price#34] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.sf100/explain.txt index 8294ae86b9b44..339d7d013972d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.sf100/explain.txt @@ -257,7 +257,7 @@ Results [8]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, count( (47) TakeOrderedAndProject Input [8]: [cd_gender#23, cd_marital_status#24, cd_education_status#25, cnt1#32, cd_purchase_estimate#26, cnt2#33, cd_credit_rating#27, cnt3#34] -Arguments: 100, 0, [cd_gender#23 ASC NULLS FIRST, cd_marital_status#24 ASC NULLS FIRST, cd_education_status#25 ASC NULLS FIRST, cd_purchase_estimate#26 ASC NULLS FIRST, cd_credit_rating#27 ASC NULLS FIRST], [cd_gender#23, cd_marital_status#24, cd_education_status#25, cnt1#32, cd_purchase_estimate#26, cnt2#33, cd_credit_rating#27, cnt3#34] +Arguments: 100, [cd_gender#23 ASC NULLS FIRST, cd_marital_status#24 ASC NULLS FIRST, cd_education_status#25 ASC NULLS FIRST, cd_purchase_estimate#26 ASC NULLS FIRST, cd_credit_rating#27 ASC NULLS FIRST], [cd_gender#23, cd_marital_status#24, cd_education_status#25, cnt1#32, cd_purchase_estimate#26, cnt2#33, cd_credit_rating#27, cnt3#34] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/explain.txt index a7a96ca9b796c..f509fa18d6971 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/explain.txt @@ -232,7 +232,7 @@ Results [8]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, count( (42) TakeOrderedAndProject Input [8]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cnt1#31, cd_purchase_estimate#24, cnt2#32, cd_credit_rating#25, cnt3#33] -Arguments: 100, 0, [cd_gender#21 ASC NULLS FIRST, cd_marital_status#22 ASC NULLS FIRST, cd_education_status#23 ASC NULLS FIRST, cd_purchase_estimate#24 ASC NULLS FIRST, cd_credit_rating#25 ASC NULLS FIRST], [cd_gender#21, cd_marital_status#22, cd_education_status#23, cnt1#31, cd_purchase_estimate#24, cnt2#32, cd_credit_rating#25, cnt3#33] +Arguments: 100, [cd_gender#21 ASC NULLS FIRST, cd_marital_status#22 ASC NULLS FIRST, cd_education_status#23 ASC NULLS FIRST, cd_purchase_estimate#24 ASC NULLS FIRST, cd_credit_rating#25 ASC NULLS FIRST], [cd_gender#21, cd_marital_status#22, cd_education_status#23, cnt1#31, cd_purchase_estimate#24, cnt2#32, cd_credit_rating#25, cnt3#33] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.sf100/explain.txt index 9e8d299758dc6..129cee0449f6a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.sf100/explain.txt @@ -167,7 +167,7 @@ Results [5]: [i_item_id#21, avg(ss_quantity#4)#40 AS agg1#44, cast((avg(Unscaled (30) TakeOrderedAndProject Input [5]: [i_item_id#21, agg1#44, agg2#45, agg3#46, agg4#47] -Arguments: 100, 0, [i_item_id#21 ASC NULLS FIRST], [i_item_id#21, agg1#44, agg2#45, agg3#46, agg4#47] +Arguments: 100, [i_item_id#21 ASC NULLS FIRST], [i_item_id#21, agg1#44, agg2#45, agg3#46, agg4#47] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/explain.txt index fff407d94e2ff..d01608cf02add 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/explain.txt @@ -167,7 +167,7 @@ Results [5]: [i_item_id#17, avg(ss_quantity#4)#40 AS agg1#44, cast((avg(Unscaled (30) TakeOrderedAndProject Input [5]: [i_item_id#17, agg1#44, agg2#45, agg3#46, agg4#47] -Arguments: 100, 0, [i_item_id#17 ASC NULLS FIRST], [i_item_id#17, agg1#44, agg2#45, agg3#46, agg4#47] +Arguments: 100, [i_item_id#17 ASC NULLS FIRST], [i_item_id#17, agg1#44, agg2#45, agg3#46, agg4#47] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/explain.txt index 10a14837d7a8b..bb0ceeabca2ac 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.sf100/explain.txt @@ -239,7 +239,7 @@ Input [8]: [total_sum#33, s_state#26, s_county#27, lochierarchy#34, _w1#35, _w2# (43) TakeOrderedAndProject Input [5]: [total_sum#33, s_state#26, s_county#27, lochierarchy#34, rank_within_parent#39] -Arguments: 100, 0, [lochierarchy#34 DESC NULLS LAST, CASE WHEN (lochierarchy#34 = 0) THEN s_state#26 END ASC NULLS FIRST, rank_within_parent#39 ASC NULLS FIRST], [total_sum#33, s_state#26, s_county#27, lochierarchy#34, rank_within_parent#39] +Arguments: 100, [lochierarchy#34 DESC NULLS LAST, CASE WHEN (lochierarchy#34 = 0) THEN s_state#26 END ASC NULLS FIRST, rank_within_parent#39 ASC NULLS FIRST], [total_sum#33, s_state#26, s_county#27, lochierarchy#34, rank_within_parent#39] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt index eeef4cb82284e..a358870666530 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt @@ -239,7 +239,7 @@ Input [8]: [total_sum#33, s_state#26, s_county#27, lochierarchy#34, _w1#35, _w2# (43) TakeOrderedAndProject Input [5]: [total_sum#33, s_state#26, s_county#27, lochierarchy#34, rank_within_parent#39] -Arguments: 100, 0, [lochierarchy#34 DESC NULLS LAST, CASE WHEN (lochierarchy#34 = 0) THEN s_state#26 END ASC NULLS FIRST, rank_within_parent#39 ASC NULLS FIRST], [total_sum#33, s_state#26, s_county#27, lochierarchy#34, rank_within_parent#39] +Arguments: 100, [lochierarchy#34 DESC NULLS LAST, CASE WHEN (lochierarchy#34 = 0) THEN s_state#26 END ASC NULLS FIRST, rank_within_parent#39 ASC NULLS FIRST], [total_sum#33, s_state#26, s_county#27, lochierarchy#34, rank_within_parent#39] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/explain.txt index f238c13c4c4e7..c6971f3ea904b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/explain.txt @@ -383,7 +383,7 @@ Results [6]: [i_item_desc#21, w_warehouse_name#33, d_week_seq#25, count(1)#46 AS (70) TakeOrderedAndProject Input [6]: [i_item_desc#21, w_warehouse_name#33, d_week_seq#25, no_promo#47, promo#48, total_cnt#49] -Arguments: 100, 0, [total_cnt#49 DESC NULLS LAST, i_item_desc#21 ASC NULLS FIRST, w_warehouse_name#33 ASC NULLS FIRST, d_week_seq#25 ASC NULLS FIRST], [i_item_desc#21, w_warehouse_name#33, d_week_seq#25, no_promo#47, promo#48, total_cnt#49] +Arguments: 100, [total_cnt#49 DESC NULLS LAST, i_item_desc#21 ASC NULLS FIRST, w_warehouse_name#33 ASC NULLS FIRST, d_week_seq#25 ASC NULLS FIRST], [i_item_desc#21, w_warehouse_name#33, d_week_seq#25, no_promo#47, promo#48, total_cnt#49] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/explain.txt index 150487e6b1052..e8671f012f8dc 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/explain.txt @@ -386,7 +386,7 @@ Results [6]: [i_item_desc#19, w_warehouse_name#16, d_week_seq#29, count(1)#46 AS (70) TakeOrderedAndProject Input [6]: [i_item_desc#19, w_warehouse_name#16, d_week_seq#29, no_promo#47, promo#48, total_cnt#49] -Arguments: 100, 0, [total_cnt#49 DESC NULLS LAST, i_item_desc#19 ASC NULLS FIRST, w_warehouse_name#16 ASC NULLS FIRST, d_week_seq#29 ASC NULLS FIRST], [i_item_desc#19, w_warehouse_name#16, d_week_seq#29, no_promo#47, promo#48, total_cnt#49] +Arguments: 100, [total_cnt#49 DESC NULLS LAST, i_item_desc#19 ASC NULLS FIRST, w_warehouse_name#16 ASC NULLS FIRST, d_week_seq#29 ASC NULLS FIRST], [i_item_desc#19, w_warehouse_name#16, d_week_seq#29, no_promo#47, promo#48, total_cnt#49] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76.sf100/explain.txt index 4f48f14edef2c..9284172139688 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76.sf100/explain.txt @@ -244,5 +244,5 @@ Results [7]: [channel#12, col_name#13, d_year#6, d_qoy#7, i_category#10, count(1 (44) TakeOrderedAndProject Input [7]: [channel#12, col_name#13, d_year#6, d_qoy#7, i_category#10, sales_cnt#48, sales_amt#49] -Arguments: 100, 0, [channel#12 ASC NULLS FIRST, col_name#13 ASC NULLS FIRST, d_year#6 ASC NULLS FIRST, d_qoy#7 ASC NULLS FIRST, i_category#10 ASC NULLS FIRST], [channel#12, col_name#13, d_year#6, d_qoy#7, i_category#10, sales_cnt#48, sales_amt#49] +Arguments: 100, [channel#12 ASC NULLS FIRST, col_name#13 ASC NULLS FIRST, d_year#6 ASC NULLS FIRST, d_qoy#7 ASC NULLS FIRST, i_category#10 ASC NULLS FIRST], [channel#12, col_name#13, d_year#6, d_qoy#7, i_category#10, sales_cnt#48, sales_amt#49] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/explain.txt index 9807ba4eccff2..026c9396cd025 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/explain.txt @@ -208,5 +208,5 @@ Results [7]: [channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6, count(1 (38) TakeOrderedAndProject Input [7]: [channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6, sales_cnt#46, sales_amt#47] -Arguments: 100, 0, [channel#12 ASC NULLS FIRST, col_name#13 ASC NULLS FIRST, d_year#9 ASC NULLS FIRST, d_qoy#10 ASC NULLS FIRST, i_category#6 ASC NULLS FIRST], [channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6, sales_cnt#46, sales_amt#47] +Arguments: 100, [channel#12 ASC NULLS FIRST, col_name#13 ASC NULLS FIRST, d_year#9 ASC NULLS FIRST, d_qoy#10 ASC NULLS FIRST, i_category#6 ASC NULLS FIRST], [channel#12, col_name#13, d_year#9, d_qoy#10, i_category#6, sales_cnt#46, sales_amt#47] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.sf100/explain.txt index 84d6f02a16373..0a398a55f7975 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.sf100/explain.txt @@ -487,7 +487,7 @@ Results [5]: [channel#105, id#106, sum(sales#16)#121 AS sales#124, sum(returns#3 (85) TakeOrderedAndProject Input [5]: [channel#105, id#106, sales#124, returns#125, profit#126] -Arguments: 100, 0, [channel#105 ASC NULLS FIRST, id#106 ASC NULLS FIRST], [channel#105, id#106, sales#124, returns#125, profit#126] +Arguments: 100, [channel#105 ASC NULLS FIRST, id#106 ASC NULLS FIRST], [channel#105, id#106, sales#124, returns#125, profit#126] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/explain.txt index 86eb206b1415f..894399b2e6061 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/explain.txt @@ -487,7 +487,7 @@ Results [5]: [channel#105, id#106, sum(sales#16)#121 AS sales#124, sum(returns#3 (85) TakeOrderedAndProject Input [5]: [channel#105, id#106, sales#124, returns#125, profit#126] -Arguments: 100, 0, [channel#105 ASC NULLS FIRST, id#106 ASC NULLS FIRST], [channel#105, id#106, sales#124, returns#125, profit#126] +Arguments: 100, [channel#105 ASC NULLS FIRST, id#106 ASC NULLS FIRST], [channel#105, id#106, sales#124, returns#125, profit#126] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79.sf100/explain.txt index b188bdb56ba24..386e889f9ef4a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79.sf100/explain.txt @@ -182,7 +182,7 @@ Input [8]: [ss_ticket_number#5, ss_customer_sk#1, s_city#17, amt#26, profit#27, (33) TakeOrderedAndProject Input [7]: [c_last_name#31, c_first_name#30, substr(s_city, 1, 30)#33, ss_ticket_number#5, amt#26, profit#27, s_city#17] -Arguments: 100, 0, [c_last_name#31 ASC NULLS FIRST, c_first_name#30 ASC NULLS FIRST, substr(s_city#17, 1, 30) ASC NULLS FIRST, profit#27 ASC NULLS FIRST], [c_last_name#31, c_first_name#30, substr(s_city, 1, 30)#33, ss_ticket_number#5, amt#26, profit#27] +Arguments: 100, [c_last_name#31 ASC NULLS FIRST, c_first_name#30 ASC NULLS FIRST, substr(s_city#17, 1, 30) ASC NULLS FIRST, profit#27 ASC NULLS FIRST], [c_last_name#31, c_first_name#30, substr(s_city, 1, 30)#33, ss_ticket_number#5, amt#26, profit#27] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79/explain.txt index 6d1ab2c4b677f..723a46f2bbcf4 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79/explain.txt @@ -167,7 +167,7 @@ Input [8]: [ss_ticket_number#5, ss_customer_sk#1, s_city#13, amt#26, profit#27, (30) TakeOrderedAndProject Input [7]: [c_last_name#30, c_first_name#29, substr(s_city, 1, 30)#32, ss_ticket_number#5, amt#26, profit#27, s_city#13] -Arguments: 100, 0, [c_last_name#30 ASC NULLS FIRST, c_first_name#29 ASC NULLS FIRST, substr(s_city#13, 1, 30) ASC NULLS FIRST, profit#27 ASC NULLS FIRST], [c_last_name#30, c_first_name#29, substr(s_city, 1, 30)#32, ss_ticket_number#5, amt#26, profit#27] +Arguments: 100, [c_last_name#30 ASC NULLS FIRST, c_first_name#29 ASC NULLS FIRST, substr(s_city#13, 1, 30) ASC NULLS FIRST, profit#27 ASC NULLS FIRST], [c_last_name#30, c_first_name#29, substr(s_city, 1, 30)#32, ss_ticket_number#5, amt#26, profit#27] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.sf100/explain.txt index 3d4b5e63923e9..e8c77e7de05eb 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.sf100/explain.txt @@ -276,7 +276,7 @@ Results [2]: [s_store_name#7, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#31 (49) TakeOrderedAndProject Input [2]: [s_store_name#7, sum(ss_net_profit)#32] -Arguments: 100, 0, [s_store_name#7 ASC NULLS FIRST], [s_store_name#7, sum(ss_net_profit)#32] +Arguments: 100, [s_store_name#7 ASC NULLS FIRST], [s_store_name#7, sum(ss_net_profit)#32] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/explain.txt index b24e9664aabdc..efac82f31fdb4 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/explain.txt @@ -246,7 +246,7 @@ Results [2]: [s_store_name#7, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#29 (43) TakeOrderedAndProject Input [2]: [s_store_name#7, sum(ss_net_profit)#30] -Arguments: 100, 0, [s_store_name#7 ASC NULLS FIRST], [s_store_name#7, sum(ss_net_profit)#30] +Arguments: 100, [s_store_name#7 ASC NULLS FIRST], [s_store_name#7, sum(ss_net_profit)#30] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.sf100/explain.txt index a542ba5b88748..26acaf66589da 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.sf100/explain.txt @@ -589,7 +589,7 @@ Results [5]: [channel#123, id#124, sum(sales#40)#139 AS sales#142, sum(returns#4 (107) TakeOrderedAndProject Input [5]: [channel#123, id#124, sales#142, returns#143, profit#144] -Arguments: 100, 0, [channel#123 ASC NULLS FIRST, id#124 ASC NULLS FIRST], [channel#123, id#124, sales#142, returns#143, profit#144] +Arguments: 100, [channel#123 ASC NULLS FIRST, id#124 ASC NULLS FIRST], [channel#123, id#124, sales#142, returns#143, profit#144] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/explain.txt index 53ec7c76b0d54..02df641cf5a20 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/explain.txt @@ -589,7 +589,7 @@ Results [5]: [channel#123, id#124, sum(sales#40)#139 AS sales#142, sum(returns#4 (107) TakeOrderedAndProject Input [5]: [channel#123, id#124, sales#142, returns#143, profit#144] -Arguments: 100, 0, [channel#123 ASC NULLS FIRST, id#124 ASC NULLS FIRST], [channel#123, id#124, sales#142, returns#143, profit#144] +Arguments: 100, [channel#123 ASC NULLS FIRST, id#124 ASC NULLS FIRST], [channel#123, id#124, sales#142, returns#143, profit#144] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.sf100/explain.txt index d7c54442bccb6..304c487dab724 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.sf100/explain.txt @@ -318,7 +318,7 @@ Input [19]: [c_customer_id#2, c_salutation#4, c_first_name#5, c_last_name#6, ca_ (57) TakeOrderedAndProject Input [16]: [c_customer_id#2, c_salutation#4, c_first_name#5, c_last_name#6, ca_street_number#8, ca_street_name#9, ca_street_type#10, ca_suite_number#11, ca_city#12, ca_county#13, ca_state#14, ca_zip#15, ca_country#16, ca_gmt_offset#17, ca_location_type#18, ctr_total_return#37] -Arguments: 100, 0, [c_customer_id#2 ASC NULLS FIRST, c_salutation#4 ASC NULLS FIRST, c_first_name#5 ASC NULLS FIRST, c_last_name#6 ASC NULLS FIRST, ca_street_number#8 ASC NULLS FIRST, ca_street_name#9 ASC NULLS FIRST, ca_street_type#10 ASC NULLS FIRST, ca_suite_number#11 ASC NULLS FIRST, ca_city#12 ASC NULLS FIRST, ca_county#13 ASC NULLS FIRST, ca_state#14 ASC NULLS FIRST, ca_zip#15 ASC NULLS FIRST, ca_country#16 ASC NULLS FIRST, ca_gmt_offset#17 ASC NULLS FIRST, ca_location_type#18 ASC NULLS FIRST, ctr_total_return#37 ASC NULLS FIRST], [c_customer_id#2, c_salutation#4, c_first_name#5, c_last_name#6, ca_street_number#8, ca_street_name#9, ca_street_type#10, ca_suite_number#11, ca_city#12, ca_county#13, ca_state#14, ca_zip#15, ca_country#16, ca_gmt_offset#17, ca_location_type#18, ctr_total_return#37] +Arguments: 100, [c_customer_id#2 ASC NULLS FIRST, c_salutation#4 ASC NULLS FIRST, c_first_name#5 ASC NULLS FIRST, c_last_name#6 ASC NULLS FIRST, ca_street_number#8 ASC NULLS FIRST, ca_street_name#9 ASC NULLS FIRST, ca_street_type#10 ASC NULLS FIRST, ca_suite_number#11 ASC NULLS FIRST, ca_city#12 ASC NULLS FIRST, ca_county#13 ASC NULLS FIRST, ca_state#14 ASC NULLS FIRST, ca_zip#15 ASC NULLS FIRST, ca_country#16 ASC NULLS FIRST, ca_gmt_offset#17 ASC NULLS FIRST, ca_location_type#18 ASC NULLS FIRST, ctr_total_return#37 ASC NULLS FIRST], [c_customer_id#2, c_salutation#4, c_first_name#5, c_last_name#6, ca_street_number#8, ca_street_name#9, ca_street_type#10, ca_suite_number#11, ca_city#12, ca_county#13, ca_state#14, ca_zip#15, ca_country#16, ca_gmt_offset#17, ca_location_type#18, ctr_total_return#37] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/explain.txt index f2a8ee58cd694..21992cf79a0ba 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/explain.txt @@ -273,7 +273,7 @@ Input [18]: [ctr_total_return#16, c_customer_id#30, c_current_addr_sk#31, c_salu (48) TakeOrderedAndProject Input [16]: [c_customer_id#30, c_salutation#32, c_first_name#33, c_last_name#34, ca_street_number#37, ca_street_name#38, ca_street_type#39, ca_suite_number#40, ca_city#41, ca_county#42, ca_state#43, ca_zip#44, ca_country#45, ca_gmt_offset#46, ca_location_type#47, ctr_total_return#16] -Arguments: 100, 0, [c_customer_id#30 ASC NULLS FIRST, c_salutation#32 ASC NULLS FIRST, c_first_name#33 ASC NULLS FIRST, c_last_name#34 ASC NULLS FIRST, ca_street_number#37 ASC NULLS FIRST, ca_street_name#38 ASC NULLS FIRST, ca_street_type#39 ASC NULLS FIRST, ca_suite_number#40 ASC NULLS FIRST, ca_city#41 ASC NULLS FIRST, ca_county#42 ASC NULLS FIRST, ca_state#43 ASC NULLS FIRST, ca_zip#44 ASC NULLS FIRST, ca_country#45 ASC NULLS FIRST, ca_gmt_offset#46 ASC NULLS FIRST, ca_location_type#47 ASC NULLS FIRST, ctr_total_return#16 ASC NULLS FIRST], [c_customer_id#30, c_salutation#32, c_first_name#33, c_last_name#34, ca_street_number#37, ca_street_name#38, ca_street_type#39, ca_suite_number#40, ca_city#41, ca_county#42, ca_state#43, ca_zip#44, ca_country#45, ca_gmt_offset#46, ca_location_type#47, ctr_total_return#16] +Arguments: 100, [c_customer_id#30 ASC NULLS FIRST, c_salutation#32 ASC NULLS FIRST, c_first_name#33 ASC NULLS FIRST, c_last_name#34 ASC NULLS FIRST, ca_street_number#37 ASC NULLS FIRST, ca_street_name#38 ASC NULLS FIRST, ca_street_type#39 ASC NULLS FIRST, ca_suite_number#40 ASC NULLS FIRST, ca_city#41 ASC NULLS FIRST, ca_county#42 ASC NULLS FIRST, ca_state#43 ASC NULLS FIRST, ca_zip#44 ASC NULLS FIRST, ca_country#45 ASC NULLS FIRST, ca_gmt_offset#46 ASC NULLS FIRST, ca_location_type#47 ASC NULLS FIRST, ctr_total_return#16 ASC NULLS FIRST], [c_customer_id#30, c_salutation#32, c_first_name#33, c_last_name#34, ca_street_number#37, ca_street_name#38, ca_street_type#39, ca_suite_number#40, ca_city#41, ca_county#42, ca_state#43, ca_zip#44, ca_country#45, ca_gmt_offset#46, ca_location_type#47, ctr_total_return#16] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.sf100/explain.txt index 5228c6f22a16d..1e0c78c7329c8 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.sf100/explain.txt @@ -154,7 +154,7 @@ Results [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] (28) TakeOrderedAndProject Input [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] -Arguments: 100, 0, [i_item_id#2 ASC NULLS FIRST], [i_item_id#2, i_item_desc#3, i_current_price#4] +Arguments: 100, [i_item_id#2 ASC NULLS FIRST], [i_item_id#2, i_item_desc#3, i_current_price#4] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/explain.txt index d9ef291bc6937..4e9ad2199c037 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/explain.txt @@ -139,7 +139,7 @@ Results [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] (25) TakeOrderedAndProject Input [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] -Arguments: 100, 0, [i_item_id#2 ASC NULLS FIRST], [i_item_id#2, i_item_desc#3, i_current_price#4] +Arguments: 100, [i_item_id#2 ASC NULLS FIRST], [i_item_id#2, i_item_desc#3, i_current_price#4] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/explain.txt index 8dbbc0b7ece92..6cac91a51f73a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/explain.txt @@ -261,7 +261,7 @@ Input [5]: [item_id#13, sr_item_qty#14, cr_item_qty#26, item_id#38, wr_item_qty# (46) TakeOrderedAndProject Input [8]: [item_id#13, sr_item_qty#14, sr_dev#41, cr_item_qty#26, cr_dev#42, wr_item_qty#39, wr_dev#43, average#44] -Arguments: 100, 0, [item_id#13 ASC NULLS FIRST, sr_item_qty#14 ASC NULLS FIRST], [item_id#13, sr_item_qty#14, sr_dev#41, cr_item_qty#26, cr_dev#42, wr_item_qty#39, wr_dev#43, average#44] +Arguments: 100, [item_id#13 ASC NULLS FIRST, sr_item_qty#14 ASC NULLS FIRST], [item_id#13, sr_item_qty#14, sr_dev#41, cr_item_qty#26, cr_dev#42, wr_item_qty#39, wr_dev#43, average#44] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/explain.txt index 19b201c2ff155..7e76d4b527ad8 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/explain.txt @@ -261,7 +261,7 @@ Input [5]: [item_id#13, sr_item_qty#14, cr_item_qty#26, item_id#38, wr_item_qty# (46) TakeOrderedAndProject Input [8]: [item_id#13, sr_item_qty#14, sr_dev#41, cr_item_qty#26, cr_dev#42, wr_item_qty#39, wr_dev#43, average#44] -Arguments: 100, 0, [item_id#13 ASC NULLS FIRST, sr_item_qty#14 ASC NULLS FIRST], [item_id#13, sr_item_qty#14, sr_dev#41, cr_item_qty#26, cr_dev#42, wr_item_qty#39, wr_dev#43, average#44] +Arguments: 100, [item_id#13 ASC NULLS FIRST, sr_item_qty#14 ASC NULLS FIRST], [item_id#13, sr_item_qty#14, sr_dev#41, cr_item_qty#26, cr_dev#42, wr_item_qty#39, wr_dev#43, average#44] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84.sf100/explain.txt index 4991ef5b3f374..9762d51e943e8 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84.sf100/explain.txt @@ -201,5 +201,5 @@ Input [5]: [c_customer_id#1, c_first_name#5, c_last_name#6, cd_demo_sk#18, sr_cd (37) TakeOrderedAndProject Input [3]: [customer_id#22, customername#23, c_customer_id#1] -Arguments: 100, 0, [c_customer_id#1 ASC NULLS FIRST], [customer_id#22, customername#23] +Arguments: 100, [c_customer_id#1 ASC NULLS FIRST], [customer_id#22, customername#23] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84/explain.txt index baf0664cd7c65..d5b84f52f4b00 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84/explain.txt @@ -201,5 +201,5 @@ Input [5]: [c_customer_id#1, c_first_name#5, c_last_name#6, cd_demo_sk#10, sr_cd (37) TakeOrderedAndProject Input [3]: [customer_id#22, customername#23, c_customer_id#1] -Arguments: 100, 0, [c_customer_id#1 ASC NULLS FIRST], [customer_id#22, customername#23] +Arguments: 100, [c_customer_id#1 ASC NULLS FIRST], [customer_id#22, customername#23] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.sf100/explain.txt index 31a12b816aee4..82f2b017b6bf5 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.sf100/explain.txt @@ -296,7 +296,7 @@ Results [4]: [substr(r_reason_desc#37, 1, 20) AS substr(r_reason_desc, 1, 20)#55 (54) TakeOrderedAndProject Input [4]: [substr(r_reason_desc, 1, 20)#55, avg(ws_quantity)#56, avg(wr_refunded_cash)#57, avg(wr_fee)#58] -Arguments: 100, 0, [substr(r_reason_desc, 1, 20)#55 ASC NULLS FIRST, avg(ws_quantity)#56 ASC NULLS FIRST, avg(wr_refunded_cash)#57 ASC NULLS FIRST, avg(wr_fee)#58 ASC NULLS FIRST], [substr(r_reason_desc, 1, 20)#55, avg(ws_quantity)#56, avg(wr_refunded_cash)#57, avg(wr_fee)#58] +Arguments: 100, [substr(r_reason_desc, 1, 20)#55 ASC NULLS FIRST, avg(ws_quantity)#56 ASC NULLS FIRST, avg(wr_refunded_cash)#57 ASC NULLS FIRST, avg(wr_fee)#58 ASC NULLS FIRST], [substr(r_reason_desc, 1, 20)#55, avg(ws_quantity)#56, avg(wr_refunded_cash)#57, avg(wr_fee)#58] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/explain.txt index 8cf66b3f6213d..6ce464ad0805e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/explain.txt @@ -266,7 +266,7 @@ Results [4]: [substr(r_reason_desc#35, 1, 20) AS substr(r_reason_desc, 1, 20)#53 (48) TakeOrderedAndProject Input [4]: [substr(r_reason_desc, 1, 20)#53, avg(ws_quantity)#54, avg(wr_refunded_cash)#55, avg(wr_fee)#56] -Arguments: 100, 0, [substr(r_reason_desc, 1, 20)#53 ASC NULLS FIRST, avg(ws_quantity)#54 ASC NULLS FIRST, avg(wr_refunded_cash)#55 ASC NULLS FIRST, avg(wr_fee)#56 ASC NULLS FIRST], [substr(r_reason_desc, 1, 20)#53, avg(ws_quantity)#54, avg(wr_refunded_cash)#55, avg(wr_fee)#56] +Arguments: 100, [substr(r_reason_desc, 1, 20)#53 ASC NULLS FIRST, avg(ws_quantity)#54 ASC NULLS FIRST, avg(wr_refunded_cash)#55 ASC NULLS FIRST, avg(wr_fee)#56 ASC NULLS FIRST], [substr(r_reason_desc, 1, 20)#53, avg(ws_quantity)#54, avg(wr_refunded_cash)#55, avg(wr_fee)#56] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.sf100/explain.txt index 76962196c8414..0cc089ebeb840 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.sf100/explain.txt @@ -116,7 +116,7 @@ Input [8]: [total_sum#17, i_category#10, i_class#11, lochierarchy#18, _w1#19, _w (21) TakeOrderedAndProject Input [5]: [total_sum#17, i_category#10, i_class#11, lochierarchy#18, rank_within_parent#23] -Arguments: 100, 0, [lochierarchy#18 DESC NULLS LAST, CASE WHEN (lochierarchy#18 = 0) THEN i_category#10 END ASC NULLS FIRST, rank_within_parent#23 ASC NULLS FIRST], [total_sum#17, i_category#10, i_class#11, lochierarchy#18, rank_within_parent#23] +Arguments: 100, [lochierarchy#18 DESC NULLS LAST, CASE WHEN (lochierarchy#18 = 0) THEN i_category#10 END ASC NULLS FIRST, rank_within_parent#23 ASC NULLS FIRST], [total_sum#17, i_category#10, i_class#11, lochierarchy#18, rank_within_parent#23] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/explain.txt index 76962196c8414..0cc089ebeb840 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/explain.txt @@ -116,7 +116,7 @@ Input [8]: [total_sum#17, i_category#10, i_class#11, lochierarchy#18, _w1#19, _w (21) TakeOrderedAndProject Input [5]: [total_sum#17, i_category#10, i_class#11, lochierarchy#18, rank_within_parent#23] -Arguments: 100, 0, [lochierarchy#18 DESC NULLS LAST, CASE WHEN (lochierarchy#18 = 0) THEN i_category#10 END ASC NULLS FIRST, rank_within_parent#23 ASC NULLS FIRST], [total_sum#17, i_category#10, i_class#11, lochierarchy#18, rank_within_parent#23] +Arguments: 100, [lochierarchy#18 DESC NULLS LAST, CASE WHEN (lochierarchy#18 = 0) THEN i_category#10 END ASC NULLS FIRST, rank_within_parent#23 ASC NULLS FIRST], [total_sum#17, i_category#10, i_class#11, lochierarchy#18, rank_within_parent#23] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.sf100/explain.txt index f2f11063b3b6f..3be73b34cb9f1 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.sf100/explain.txt @@ -149,7 +149,7 @@ Input [9]: [i_category#4, i_class#3, i_brand#2, s_store_name#14, s_company_name# (27) TakeOrderedAndProject Input [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#14, s_company_name#15, d_moy#12, sum_sales#21, avg_monthly_sales#24] -Arguments: 100, 0, [(sum_sales#21 - avg_monthly_sales#24) ASC NULLS FIRST, s_store_name#14 ASC NULLS FIRST], [i_category#4, i_class#3, i_brand#2, s_store_name#14, s_company_name#15, d_moy#12, sum_sales#21, avg_monthly_sales#24] +Arguments: 100, [(sum_sales#21 - avg_monthly_sales#24) ASC NULLS FIRST, s_store_name#14 ASC NULLS FIRST], [i_category#4, i_class#3, i_brand#2, s_store_name#14, s_company_name#15, d_moy#12, sum_sales#21, avg_monthly_sales#24] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/explain.txt index 472c590e5518c..5a572dc71127f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/explain.txt @@ -149,7 +149,7 @@ Input [9]: [i_category#4, i_class#3, i_brand#2, s_store_name#14, s_company_name# (27) TakeOrderedAndProject Input [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#14, s_company_name#15, d_moy#12, sum_sales#21, avg_monthly_sales#24] -Arguments: 100, 0, [(sum_sales#21 - avg_monthly_sales#24) ASC NULLS FIRST, s_store_name#14 ASC NULLS FIRST], [i_category#4, i_class#3, i_brand#2, s_store_name#14, s_company_name#15, d_moy#12, sum_sales#21, avg_monthly_sales#24] +Arguments: 100, [(sum_sales#21 - avg_monthly_sales#24) ASC NULLS FIRST, s_store_name#14 ASC NULLS FIRST], [i_category#4, i_class#3, i_brand#2, s_store_name#14, s_company_name#15, d_moy#12, sum_sales#21, avg_monthly_sales#24] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.sf100/explain.txt index 07d897082d46c..57cc6bfc9ae23 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.sf100/explain.txt @@ -132,5 +132,5 @@ Results [2]: [ss_customer_sk#11, sum(act_sales#17)#23 AS sumsales#24] (24) TakeOrderedAndProject Input [2]: [ss_customer_sk#11, sumsales#24] -Arguments: 100, 0, [sumsales#24 ASC NULLS FIRST, ss_customer_sk#11 ASC NULLS FIRST], [ss_customer_sk#11, sumsales#24] +Arguments: 100, [sumsales#24 ASC NULLS FIRST, ss_customer_sk#11 ASC NULLS FIRST], [ss_customer_sk#11, sumsales#24] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/explain.txt index f79a19dec0dbb..06f693cc164c0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/explain.txt @@ -132,5 +132,5 @@ Results [2]: [ss_customer_sk#2, sum(act_sales#17)#23 AS sumsales#24] (24) TakeOrderedAndProject Input [2]: [ss_customer_sk#2, sumsales#24] -Arguments: 100, 0, [sumsales#24 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST], [ss_customer_sk#2, sumsales#24] +Arguments: 100, [sumsales#24 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST], [ss_customer_sk#2, sumsales#24] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99.sf100/explain.txt index 5678826cdde44..1b955ee3bd96c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99.sf100/explain.txt @@ -179,5 +179,5 @@ Results [8]: [_groupingexpression#18 AS substr(w_warehouse_name, 1, 20)#35, sm_t (32) TakeOrderedAndProject Input [8]: [substr(w_warehouse_name, 1, 20)#35, sm_type#10, cc_name#13, 30 days #36, 31 - 60 days #37, 61 - 90 days #38, 91 - 120 days #39, >120 days #40] -Arguments: 100, 0, [substr(w_warehouse_name, 1, 20)#35 ASC NULLS FIRST, sm_type#10 ASC NULLS FIRST, cc_name#13 ASC NULLS FIRST], [substr(w_warehouse_name, 1, 20)#35, sm_type#10, cc_name#13, 30 days #36, 31 - 60 days #37, 61 - 90 days #38, 91 - 120 days #39, >120 days #40] +Arguments: 100, [substr(w_warehouse_name, 1, 20)#35 ASC NULLS FIRST, sm_type#10 ASC NULLS FIRST, cc_name#13 ASC NULLS FIRST], [substr(w_warehouse_name, 1, 20)#35, sm_type#10, cc_name#13, 30 days #36, 31 - 60 days #37, 61 - 90 days #38, 91 - 120 days #39, >120 days #40] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99/explain.txt index 0c6114d1cbd91..1431623539828 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99/explain.txt @@ -179,5 +179,5 @@ Results [8]: [_groupingexpression#18 AS substr(w_warehouse_name, 1, 20)#35, sm_t (32) TakeOrderedAndProject Input [8]: [substr(w_warehouse_name, 1, 20)#35, sm_type#10, cc_name#13, 30 days #36, 31 - 60 days #37, 61 - 90 days #38, 91 - 120 days #39, >120 days #40] -Arguments: 100, 0, [substr(w_warehouse_name, 1, 20)#35 ASC NULLS FIRST, sm_type#10 ASC NULLS FIRST, cc_name#13 ASC NULLS FIRST], [substr(w_warehouse_name, 1, 20)#35, sm_type#10, cc_name#13, 30 days #36, 31 - 60 days #37, 61 - 90 days #38, 91 - 120 days #39, >120 days #40] +Arguments: 100, [substr(w_warehouse_name, 1, 20)#35 ASC NULLS FIRST, sm_type#10 ASC NULLS FIRST, cc_name#13 ASC NULLS FIRST], [substr(w_warehouse_name, 1, 20)#35, sm_type#10, cc_name#13, 30 days #36, 31 - 60 days #37, 61 - 90 days #38, 91 - 120 days #39, >120 days #40] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.sf100/explain.txt index ef1337b7389a1..fe97109236cf2 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.sf100/explain.txt @@ -244,7 +244,7 @@ Results [14]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, count (45) TakeOrderedAndProject Input [14]: [cd_gender#24, cd_marital_status#25, cd_education_status#26, cnt1#36, cd_purchase_estimate#27, cnt2#37, cd_credit_rating#28, cnt3#38, cd_dep_count#29, cnt4#39, cd_dep_employed_count#30, cnt5#40, cd_dep_college_count#31, cnt6#41] -Arguments: 100, 0, [cd_gender#24 ASC NULLS FIRST, cd_marital_status#25 ASC NULLS FIRST, cd_education_status#26 ASC NULLS FIRST, cd_purchase_estimate#27 ASC NULLS FIRST, cd_credit_rating#28 ASC NULLS FIRST, cd_dep_count#29 ASC NULLS FIRST, cd_dep_employed_count#30 ASC NULLS FIRST, cd_dep_college_count#31 ASC NULLS FIRST], [cd_gender#24, cd_marital_status#25, cd_education_status#26, cnt1#36, cd_purchase_estimate#27, cnt2#37, cd_credit_rating#28, cnt3#38, cd_dep_count#29, cnt4#39, cd_dep_employed_count#30, cnt5#40, cd_dep_college_count#31, cnt6#41] +Arguments: 100, [cd_gender#24 ASC NULLS FIRST, cd_marital_status#25 ASC NULLS FIRST, cd_education_status#26 ASC NULLS FIRST, cd_purchase_estimate#27 ASC NULLS FIRST, cd_credit_rating#28 ASC NULLS FIRST, cd_dep_count#29 ASC NULLS FIRST, cd_dep_employed_count#30 ASC NULLS FIRST, cd_dep_college_count#31 ASC NULLS FIRST], [cd_gender#24, cd_marital_status#25, cd_education_status#26, cnt1#36, cd_purchase_estimate#27, cnt2#37, cd_credit_rating#28, cnt3#38, cd_dep_count#29, cnt4#39, cd_dep_employed_count#30, cnt5#40, cd_dep_college_count#31, cnt6#41] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/explain.txt index 2c598e386e1b0..02522cd257d71 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/explain.txt @@ -224,7 +224,7 @@ Results [14]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, count (41) TakeOrderedAndProject Input [14]: [cd_gender#22, cd_marital_status#23, cd_education_status#24, cnt1#35, cd_purchase_estimate#25, cnt2#36, cd_credit_rating#26, cnt3#37, cd_dep_count#27, cnt4#38, cd_dep_employed_count#28, cnt5#39, cd_dep_college_count#29, cnt6#40] -Arguments: 100, 0, [cd_gender#22 ASC NULLS FIRST, cd_marital_status#23 ASC NULLS FIRST, cd_education_status#24 ASC NULLS FIRST, cd_purchase_estimate#25 ASC NULLS FIRST, cd_credit_rating#26 ASC NULLS FIRST, cd_dep_count#27 ASC NULLS FIRST, cd_dep_employed_count#28 ASC NULLS FIRST, cd_dep_college_count#29 ASC NULLS FIRST], [cd_gender#22, cd_marital_status#23, cd_education_status#24, cnt1#35, cd_purchase_estimate#25, cnt2#36, cd_credit_rating#26, cnt3#37, cd_dep_count#27, cnt4#38, cd_dep_employed_count#28, cnt5#39, cd_dep_college_count#29, cnt6#40] +Arguments: 100, [cd_gender#22 ASC NULLS FIRST, cd_marital_status#23 ASC NULLS FIRST, cd_education_status#24 ASC NULLS FIRST, cd_purchase_estimate#25 ASC NULLS FIRST, cd_credit_rating#26 ASC NULLS FIRST, cd_dep_count#27 ASC NULLS FIRST, cd_dep_employed_count#28 ASC NULLS FIRST, cd_dep_college_count#29 ASC NULLS FIRST], [cd_gender#22, cd_marital_status#23, cd_education_status#24, cnt1#35, cd_purchase_estimate#25, cnt2#36, cd_credit_rating#26, cnt3#37, cd_dep_count#27, cnt4#38, cd_dep_employed_count#28, cnt5#39, cd_dep_college_count#29, cnt6#40] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.sf100/explain.txt index 76fa78e508d06..a01e3ea970a41 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.sf100/explain.txt @@ -441,7 +441,7 @@ Input [10]: [customer_id#22, year_total#23, customer_id#44, customer_first_name# (80) TakeOrderedAndProject Input [4]: [customer_id#44, customer_first_name#45, customer_last_name#46, customer_email_address#47] -Arguments: 100, 0, [customer_id#44 ASC NULLS FIRST, customer_first_name#45 ASC NULLS FIRST, customer_last_name#46 ASC NULLS FIRST, customer_email_address#47 ASC NULLS FIRST], [customer_id#44, customer_first_name#45, customer_last_name#46, customer_email_address#47] +Arguments: 100, [customer_id#44 ASC NULLS FIRST, customer_first_name#45 ASC NULLS FIRST, customer_last_name#46 ASC NULLS FIRST, customer_email_address#47 ASC NULLS FIRST], [customer_id#44, customer_first_name#45, customer_last_name#46, customer_email_address#47] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/explain.txt index 6b45c59713742..3b77e772600e0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/explain.txt @@ -410,7 +410,7 @@ Input [10]: [customer_id#21, year_total#22, customer_id#42, customer_first_name# (72) TakeOrderedAndProject Input [4]: [customer_id#42, customer_first_name#43, customer_last_name#44, customer_email_address#45] -Arguments: 100, 0, [customer_id#42 ASC NULLS FIRST, customer_first_name#43 ASC NULLS FIRST, customer_last_name#44 ASC NULLS FIRST, customer_email_address#45 ASC NULLS FIRST], [customer_id#42, customer_first_name#43, customer_last_name#44, customer_email_address#45] +Arguments: 100, [customer_id#42 ASC NULLS FIRST, customer_first_name#43 ASC NULLS FIRST, customer_last_name#44 ASC NULLS FIRST, customer_email_address#45 ASC NULLS FIRST], [customer_id#42, customer_first_name#43, customer_last_name#44, customer_email_address#45] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.sf100/explain.txt index b011014d38f09..8cdf63ac2c420 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.sf100/explain.txt @@ -126,7 +126,7 @@ Input [9]: [i_item_id#7, i_item_desc#8, i_category#11, i_class#10, i_current_pri (23) TakeOrderedAndProject Input [7]: [i_item_id#7, i_item_desc#8, i_category#11, i_class#10, i_current_price#9, itemrevenue#18, revenueratio#23] -Arguments: 100, 0, [i_category#11 ASC NULLS FIRST, i_class#10 ASC NULLS FIRST, i_item_id#7 ASC NULLS FIRST, i_item_desc#8 ASC NULLS FIRST, revenueratio#23 ASC NULLS FIRST], [i_item_id#7, i_item_desc#8, i_category#11, i_class#10, i_current_price#9, itemrevenue#18, revenueratio#23] +Arguments: 100, [i_category#11 ASC NULLS FIRST, i_class#10 ASC NULLS FIRST, i_item_id#7 ASC NULLS FIRST, i_item_desc#8 ASC NULLS FIRST, revenueratio#23 ASC NULLS FIRST], [i_item_id#7, i_item_desc#8, i_category#11, i_class#10, i_current_price#9, itemrevenue#18, revenueratio#23] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/explain.txt index 7f0cad457a44c..4eac76d9fc5c1 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/explain.txt @@ -111,7 +111,7 @@ Input [9]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_pric (20) TakeOrderedAndProject Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#17, revenueratio#22] -Arguments: 100, 0, [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#22 ASC NULLS FIRST], [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#17, revenueratio#22] +Arguments: 100, [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#22 ASC NULLS FIRST], [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#17, revenueratio#22] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/explain.txt index b983676a12f24..25791c2f6ec4d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/explain.txt @@ -564,7 +564,7 @@ Join condition: None (102) TakeOrderedAndProject Input [12]: [channel#63, i_brand_id#49, i_class_id#50, i_category_id#51, sales#64, number_sales#65, channel#88, i_brand_id#76, i_class_id#77, i_category_id#78, sales#89, number_sales#90] -Arguments: 100, 0, [i_brand_id#49 ASC NULLS FIRST, i_class_id#50 ASC NULLS FIRST, i_category_id#51 ASC NULLS FIRST], [channel#63, i_brand_id#49, i_class_id#50, i_category_id#51, sales#64, number_sales#65, channel#88, i_brand_id#76, i_class_id#77, i_category_id#78, sales#89, number_sales#90] +Arguments: 100, [i_brand_id#49 ASC NULLS FIRST, i_class_id#50 ASC NULLS FIRST, i_category_id#51 ASC NULLS FIRST], [channel#63, i_brand_id#49, i_class_id#50, i_category_id#51, sales#64, number_sales#65, channel#88, i_brand_id#76, i_class_id#77, i_category_id#78, sales#89, number_sales#90] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt index 385c4861d3a53..aaf1bedf681b8 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt @@ -484,7 +484,7 @@ Join condition: None (86) TakeOrderedAndProject Input [12]: [channel#58, i_brand_id#44, i_class_id#45, i_category_id#46, sales#59, number_sales#60, channel#82, i_brand_id#69, i_class_id#70, i_category_id#71, sales#83, number_sales#84] -Arguments: 100, 0, [i_brand_id#44 ASC NULLS FIRST, i_class_id#45 ASC NULLS FIRST, i_category_id#46 ASC NULLS FIRST], [channel#58, i_brand_id#44, i_class_id#45, i_category_id#46, sales#59, number_sales#60, channel#82, i_brand_id#69, i_class_id#70, i_category_id#71, sales#83, number_sales#84] +Arguments: 100, [i_brand_id#44 ASC NULLS FIRST, i_class_id#45 ASC NULLS FIRST, i_category_id#46 ASC NULLS FIRST], [channel#58, i_brand_id#44, i_class_id#45, i_category_id#46, sales#59, number_sales#60, channel#82, i_brand_id#69, i_class_id#70, i_category_id#71, sales#83, number_sales#84] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/explain.txt index cddfda97aeb20..daa4bcfdfc264 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/explain.txt @@ -832,7 +832,7 @@ Results [6]: [channel#63, i_brand_id#49, i_class_id#50, i_category_id#51, sum_sa (146) TakeOrderedAndProject Input [6]: [channel#63, i_brand_id#49, i_class_id#50, i_category_id#51, sum_sales#121, number_sales#122] -Arguments: 100, 0, [channel#63 ASC NULLS FIRST, i_brand_id#49 ASC NULLS FIRST, i_class_id#50 ASC NULLS FIRST, i_category_id#51 ASC NULLS FIRST], [channel#63, i_brand_id#49, i_class_id#50, i_category_id#51, sum_sales#121, number_sales#122] +Arguments: 100, [channel#63 ASC NULLS FIRST, i_brand_id#49 ASC NULLS FIRST, i_class_id#50 ASC NULLS FIRST, i_category_id#51 ASC NULLS FIRST], [channel#63, i_brand_id#49, i_class_id#50, i_category_id#51, sum_sales#121, number_sales#122] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/explain.txt index 303aab65a4ab9..171318a3df7db 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/explain.txt @@ -737,7 +737,7 @@ Results [6]: [channel#58, i_brand_id#44, i_class_id#45, i_category_id#46, sum_sa (127) TakeOrderedAndProject Input [6]: [channel#58, i_brand_id#44, i_class_id#45, i_category_id#46, sum_sales#114, number_sales#115] -Arguments: 100, 0, [channel#58 ASC NULLS FIRST, i_brand_id#44 ASC NULLS FIRST, i_class_id#45 ASC NULLS FIRST, i_category_id#46 ASC NULLS FIRST], [channel#58, i_brand_id#44, i_class_id#45, i_category_id#46, sum_sales#114, number_sales#115] +Arguments: 100, [channel#58 ASC NULLS FIRST, i_brand_id#44 ASC NULLS FIRST, i_class_id#45 ASC NULLS FIRST, i_category_id#46 ASC NULLS FIRST], [channel#58, i_brand_id#44, i_class_id#45, i_category_id#46, sum_sales#114, number_sales#115] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.sf100/explain.txt index 012bbaa8e45c3..506e18eabcc20 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.sf100/explain.txt @@ -853,7 +853,7 @@ Results [11]: [null AS i_item_id#266, null AS ca_country#267, null AS ca_state#2 (156) TakeOrderedAndProject Input [11]: [i_item_id#18, ca_country#29, ca_state#28, ca_county#27, agg1#78, agg2#79, agg3#80, agg4#81, agg5#82, agg6#83, agg7#84] -Arguments: 100, 0, [ca_country#29 ASC NULLS FIRST, ca_state#28 ASC NULLS FIRST, ca_county#27 ASC NULLS FIRST, i_item_id#18 ASC NULLS FIRST], [i_item_id#18, ca_country#29, ca_state#28, ca_county#27, agg1#78, agg2#79, agg3#80, agg4#81, agg5#82, agg6#83, agg7#84] +Arguments: 100, [ca_country#29 ASC NULLS FIRST, ca_state#28 ASC NULLS FIRST, ca_county#27 ASC NULLS FIRST, i_item_id#18 ASC NULLS FIRST], [i_item_id#18, ca_country#29, ca_state#28, ca_county#27, agg1#78, agg2#79, agg3#80, agg4#81, agg5#82, agg6#83, agg7#84] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a/explain.txt index fdfa28cf8ee18..7bd7a7ae80176 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a/explain.txt @@ -834,7 +834,7 @@ Results [11]: [null AS i_item_id#258, null AS ca_country#259, null AS ca_state#2 (153) TakeOrderedAndProject Input [11]: [i_item_id#31, ca_country#27, ca_state#26, ca_county#25, agg1#76, agg2#77, agg3#78, agg4#79, agg5#80, agg6#81, agg7#82] -Arguments: 100, 0, [ca_country#27 ASC NULLS FIRST, ca_state#26 ASC NULLS FIRST, ca_county#25 ASC NULLS FIRST, i_item_id#31 ASC NULLS FIRST], [i_item_id#31, ca_country#27, ca_state#26, ca_county#25, agg1#76, agg2#77, agg3#78, agg4#79, agg5#80, agg6#81, agg7#82] +Arguments: 100, [ca_country#27 ASC NULLS FIRST, ca_state#26 ASC NULLS FIRST, ca_county#25 ASC NULLS FIRST, i_item_id#31 ASC NULLS FIRST], [i_item_id#31, ca_country#27, ca_state#26, ca_county#25, agg1#76, agg2#77, agg3#78, agg4#79, agg5#80, agg6#81, agg7#82] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.sf100/explain.txt index 8f10eecb293fb..7306778e34b12 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.sf100/explain.txt @@ -126,7 +126,7 @@ Input [9]: [i_item_id#7, i_item_desc#8, i_category#11, i_class#10, i_current_pri (23) TakeOrderedAndProject Input [7]: [i_item_id#7, i_item_desc#8, i_category#11, i_class#10, i_current_price#9, itemrevenue#18, revenueratio#23] -Arguments: 100, 0, [i_category#11 ASC NULLS FIRST, i_class#10 ASC NULLS FIRST, i_item_id#7 ASC NULLS FIRST, i_item_desc#8 ASC NULLS FIRST, revenueratio#23 ASC NULLS FIRST], [i_item_id#7, i_item_desc#8, i_category#11, i_class#10, i_current_price#9, itemrevenue#18, revenueratio#23] +Arguments: 100, [i_category#11 ASC NULLS FIRST, i_class#10 ASC NULLS FIRST, i_item_id#7 ASC NULLS FIRST, i_item_desc#8 ASC NULLS FIRST, revenueratio#23 ASC NULLS FIRST], [i_item_id#7, i_item_desc#8, i_category#11, i_class#10, i_current_price#9, itemrevenue#18, revenueratio#23] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/explain.txt index 3a1cae03eca2c..e392ff69cffe5 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/explain.txt @@ -111,7 +111,7 @@ Input [9]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_pric (20) TakeOrderedAndProject Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#17, revenueratio#22] -Arguments: 100, 0, [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#22 ASC NULLS FIRST], [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#17, revenueratio#22] +Arguments: 100, [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#22 ASC NULLS FIRST], [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#17, revenueratio#22] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22.sf100/explain.txt index fdd92d584c472..eb956b9e75ab0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22.sf100/explain.txt @@ -136,7 +136,7 @@ Results [5]: [i_product_name#14, i_brand#15, i_class#16, i_category#17, avg(inv_ (25) TakeOrderedAndProject Input [5]: [i_product_name#14, i_brand#15, i_class#16, i_category#17, qoh#25] -Arguments: 100, 0, [qoh#25 ASC NULLS FIRST, i_product_name#14 ASC NULLS FIRST, i_brand#15 ASC NULLS FIRST, i_class#16 ASC NULLS FIRST, i_category#17 ASC NULLS FIRST], [i_product_name#14, i_brand#15, i_class#16, i_category#17, qoh#25] +Arguments: 100, [qoh#25 ASC NULLS FIRST, i_product_name#14 ASC NULLS FIRST, i_brand#15 ASC NULLS FIRST, i_class#16 ASC NULLS FIRST, i_category#17 ASC NULLS FIRST], [i_product_name#14, i_brand#15, i_class#16, i_category#17, qoh#25] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/explain.txt index f912a014c28f8..85c21eca87544 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/explain.txt @@ -121,7 +121,7 @@ Results [5]: [i_product_name#13, i_brand#14, i_class#15, i_category#16, avg(inv_ (22) TakeOrderedAndProject Input [5]: [i_product_name#13, i_brand#14, i_class#15, i_category#16, qoh#24] -Arguments: 100, 0, [qoh#24 ASC NULLS FIRST, i_product_name#13 ASC NULLS FIRST, i_brand#14 ASC NULLS FIRST, i_class#15 ASC NULLS FIRST, i_category#16 ASC NULLS FIRST], [i_product_name#13, i_brand#14, i_class#15, i_category#16, qoh#24] +Arguments: 100, [qoh#24 ASC NULLS FIRST, i_product_name#13 ASC NULLS FIRST, i_brand#14 ASC NULLS FIRST, i_class#15 ASC NULLS FIRST, i_category#16 ASC NULLS FIRST], [i_product_name#13, i_brand#14, i_class#15, i_category#16, qoh#24] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.sf100/explain.txt index 58f4ab1df1e4a..e7072101f8f23 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.sf100/explain.txt @@ -290,7 +290,7 @@ Results [5]: [null AS i_product_name#62, null AS i_brand#63, null AS i_class#64, (48) TakeOrderedAndProject Input [5]: [i_product_name#14, i_brand#11, i_class#12, i_category#13, qoh#28] -Arguments: 100, 0, [qoh#28 ASC NULLS FIRST, i_product_name#14 ASC NULLS FIRST, i_brand#11 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_category#13 ASC NULLS FIRST], [i_product_name#14, i_brand#11, i_class#12, i_category#13, qoh#28] +Arguments: 100, [qoh#28 ASC NULLS FIRST, i_product_name#14 ASC NULLS FIRST, i_brand#11 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_category#13 ASC NULLS FIRST], [i_product_name#14, i_brand#11, i_class#12, i_category#13, qoh#28] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/explain.txt index d5d267ed52c3c..c5bef0d13db91 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/explain.txt @@ -275,7 +275,7 @@ Results [5]: [null AS i_product_name#61, null AS i_brand#62, null AS i_class#63, (45) TakeOrderedAndProject Input [5]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, qoh#27] -Arguments: 100, 0, [qoh#27 ASC NULLS FIRST, i_product_name#11 ASC NULLS FIRST, i_brand#8 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_category#10 ASC NULLS FIRST], [i_product_name#11, i_brand#8, i_class#9, i_category#10, qoh#27] +Arguments: 100, [qoh#27 ASC NULLS FIRST, i_product_name#11 ASC NULLS FIRST, i_brand#8 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_category#10 ASC NULLS FIRST], [i_product_name#11, i_brand#8, i_class#9, i_category#10, qoh#27] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.sf100/explain.txt index 69d7b8972b7cc..b55e5641a679a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.sf100/explain.txt @@ -404,7 +404,7 @@ Results [7]: [null AS i_item_id#102, null AS s_state#103, 1 AS g_state#104, avg( (73) TakeOrderedAndProject Input [7]: [i_item_id#20, s_state#17, g_state#47, agg1#48, agg2#49, agg3#50, agg4#51] -Arguments: 100, 0, [i_item_id#20 ASC NULLS FIRST, s_state#17 ASC NULLS FIRST], [i_item_id#20, s_state#17, g_state#47, agg1#48, agg2#49, agg3#50, agg4#51] +Arguments: 100, [i_item_id#20 ASC NULLS FIRST, s_state#17 ASC NULLS FIRST], [i_item_id#20, s_state#17, g_state#47, agg1#48, agg2#49, agg3#50, agg4#51] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a/explain.txt index 515003801461b..60b1498c4e6d0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a/explain.txt @@ -404,7 +404,7 @@ Results [7]: [null AS i_item_id#102, null AS s_state#103, 1 AS g_state#104, avg( (73) TakeOrderedAndProject Input [7]: [i_item_id#20, s_state#17, g_state#47, agg1#48, agg2#49, agg3#50, agg4#51] -Arguments: 100, 0, [i_item_id#20 ASC NULLS FIRST, s_state#17 ASC NULLS FIRST], [i_item_id#20, s_state#17, g_state#47, agg1#48, agg2#49, agg3#50, agg4#51] +Arguments: 100, [i_item_id#20 ASC NULLS FIRST, s_state#17 ASC NULLS FIRST], [i_item_id#20, s_state#17, g_state#47, agg1#48, agg2#49, agg3#50, agg4#51] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.sf100/explain.txt index 55eea4d3b5c4b..ba6ce3011207d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.sf100/explain.txt @@ -287,7 +287,7 @@ Results [18]: [ca_state#22, cd_gender#26, cd_marital_status#27, cd_dep_count#28, (53) TakeOrderedAndProject Input [18]: [ca_state#22, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cnt1#69, avg(cd_dep_count)#70, max(cd_dep_count)#71, sum(cd_dep_count)#72, cd_dep_employed_count#29, cnt2#73, avg(cd_dep_employed_count)#74, max(cd_dep_employed_count)#75, sum(cd_dep_employed_count)#76, cd_dep_college_count#30, cnt3#77, avg(cd_dep_college_count)#78, max(cd_dep_college_count)#79, sum(cd_dep_college_count)#80] -Arguments: 100, 0, [ca_state#22 ASC NULLS FIRST, cd_gender#26 ASC NULLS FIRST, cd_marital_status#27 ASC NULLS FIRST, cd_dep_count#28 ASC NULLS FIRST, cd_dep_employed_count#29 ASC NULLS FIRST, cd_dep_college_count#30 ASC NULLS FIRST], [ca_state#22, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cnt1#69, avg(cd_dep_count)#70, max(cd_dep_count)#71, sum(cd_dep_count)#72, cd_dep_employed_count#29, cnt2#73, avg(cd_dep_employed_count)#74, max(cd_dep_employed_count)#75, sum(cd_dep_employed_count)#76, cd_dep_college_count#30, cnt3#77, avg(cd_dep_college_count)#78, max(cd_dep_college_count)#79, sum(cd_dep_college_count)#80] +Arguments: 100, [ca_state#22 ASC NULLS FIRST, cd_gender#26 ASC NULLS FIRST, cd_marital_status#27 ASC NULLS FIRST, cd_dep_count#28 ASC NULLS FIRST, cd_dep_employed_count#29 ASC NULLS FIRST, cd_dep_college_count#30 ASC NULLS FIRST], [ca_state#22, cd_gender#26, cd_marital_status#27, cd_dep_count#28, cnt1#69, avg(cd_dep_count)#70, max(cd_dep_count)#71, sum(cd_dep_count)#72, cd_dep_employed_count#29, cnt2#73, avg(cd_dep_employed_count)#74, max(cd_dep_employed_count)#75, sum(cd_dep_employed_count)#76, cd_dep_college_count#30, cnt3#77, avg(cd_dep_college_count)#78, max(cd_dep_college_count)#79, sum(cd_dep_college_count)#80] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/explain.txt index c383409e22b74..a7d47450807a7 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/explain.txt @@ -232,7 +232,7 @@ Results [18]: [ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, (42) TakeOrderedAndProject Input [18]: [ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cnt1#66, avg(cd_dep_count)#67, max(cd_dep_count)#68, sum(cd_dep_count)#69, cd_dep_employed_count#26, cnt2#70, avg(cd_dep_employed_count)#71, max(cd_dep_employed_count)#72, sum(cd_dep_employed_count)#73, cd_dep_college_count#27, cnt3#74, avg(cd_dep_college_count)#75, max(cd_dep_college_count)#76, sum(cd_dep_college_count)#77] -Arguments: 100, 0, [ca_state#20 ASC NULLS FIRST, cd_gender#23 ASC NULLS FIRST, cd_marital_status#24 ASC NULLS FIRST, cd_dep_count#25 ASC NULLS FIRST, cd_dep_employed_count#26 ASC NULLS FIRST, cd_dep_college_count#27 ASC NULLS FIRST], [ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cnt1#66, avg(cd_dep_count)#67, max(cd_dep_count)#68, sum(cd_dep_count)#69, cd_dep_employed_count#26, cnt2#70, avg(cd_dep_employed_count)#71, max(cd_dep_employed_count)#72, sum(cd_dep_employed_count)#73, cd_dep_college_count#27, cnt3#74, avg(cd_dep_college_count)#75, max(cd_dep_college_count)#76, sum(cd_dep_college_count)#77] +Arguments: 100, [ca_state#20 ASC NULLS FIRST, cd_gender#23 ASC NULLS FIRST, cd_marital_status#24 ASC NULLS FIRST, cd_dep_count#25 ASC NULLS FIRST, cd_dep_employed_count#26 ASC NULLS FIRST, cd_dep_college_count#27 ASC NULLS FIRST], [ca_state#20, cd_gender#23, cd_marital_status#24, cd_dep_count#25, cnt1#66, avg(cd_dep_count)#67, max(cd_dep_count)#68, sum(cd_dep_count)#69, cd_dep_employed_count#26, cnt2#70, avg(cd_dep_employed_count)#71, max(cd_dep_employed_count)#72, sum(cd_dep_employed_count)#73, cd_dep_college_count#27, cnt3#74, avg(cd_dep_college_count)#75, max(cd_dep_college_count)#76, sum(cd_dep_college_count)#77] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.sf100/explain.txt index 9bbb67475730a..f8d8946e2af92 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.sf100/explain.txt @@ -269,7 +269,7 @@ Results [18]: [ca_state#21, cd_gender#25, cd_marital_status#26, cd_dep_count#27, (50) TakeOrderedAndProject Input [18]: [ca_state#21, cd_gender#25, cd_marital_status#26, cd_dep_count#27, cnt1#68, avg(cd_dep_count)#69, max(cd_dep_count)#70, sum(cd_dep_count)#71, cd_dep_employed_count#28, cnt2#72, avg(cd_dep_employed_count)#73, max(cd_dep_employed_count)#74, sum(cd_dep_employed_count)#75, cd_dep_college_count#29, cnt3#76, avg(cd_dep_college_count)#77, max(cd_dep_college_count)#78, sum(cd_dep_college_count)#79] -Arguments: 100, 0, [ca_state#21 ASC NULLS FIRST, cd_gender#25 ASC NULLS FIRST, cd_marital_status#26 ASC NULLS FIRST, cd_dep_count#27 ASC NULLS FIRST, cd_dep_employed_count#28 ASC NULLS FIRST, cd_dep_college_count#29 ASC NULLS FIRST], [ca_state#21, cd_gender#25, cd_marital_status#26, cd_dep_count#27, cnt1#68, avg(cd_dep_count)#69, max(cd_dep_count)#70, sum(cd_dep_count)#71, cd_dep_employed_count#28, cnt2#72, avg(cd_dep_employed_count)#73, max(cd_dep_employed_count)#74, sum(cd_dep_employed_count)#75, cd_dep_college_count#29, cnt3#76, avg(cd_dep_college_count)#77, max(cd_dep_college_count)#78, sum(cd_dep_college_count)#79] +Arguments: 100, [ca_state#21 ASC NULLS FIRST, cd_gender#25 ASC NULLS FIRST, cd_marital_status#26 ASC NULLS FIRST, cd_dep_count#27 ASC NULLS FIRST, cd_dep_employed_count#28 ASC NULLS FIRST, cd_dep_college_count#29 ASC NULLS FIRST], [ca_state#21, cd_gender#25, cd_marital_status#26, cd_dep_count#27, cnt1#68, avg(cd_dep_count)#69, max(cd_dep_count)#70, sum(cd_dep_count)#71, cd_dep_employed_count#28, cnt2#72, avg(cd_dep_employed_count)#73, max(cd_dep_employed_count)#74, sum(cd_dep_employed_count)#75, cd_dep_college_count#29, cnt3#76, avg(cd_dep_college_count)#77, max(cd_dep_college_count)#78, sum(cd_dep_college_count)#79] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/explain.txt index 3119516f9d6ff..fe6d15a3fb15b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/explain.txt @@ -219,7 +219,7 @@ Results [18]: [ca_state#19, cd_gender#22, cd_marital_status#23, cd_dep_count#24, (40) TakeOrderedAndProject Input [18]: [ca_state#19, cd_gender#22, cd_marital_status#23, cd_dep_count#24, cnt1#65, avg(cd_dep_count)#66, max(cd_dep_count)#67, sum(cd_dep_count)#68, cd_dep_employed_count#25, cnt2#69, avg(cd_dep_employed_count)#70, max(cd_dep_employed_count)#71, sum(cd_dep_employed_count)#72, cd_dep_college_count#26, cnt3#73, avg(cd_dep_college_count)#74, max(cd_dep_college_count)#75, sum(cd_dep_college_count)#76] -Arguments: 100, 0, [ca_state#19 ASC NULLS FIRST, cd_gender#22 ASC NULLS FIRST, cd_marital_status#23 ASC NULLS FIRST, cd_dep_count#24 ASC NULLS FIRST, cd_dep_employed_count#25 ASC NULLS FIRST, cd_dep_college_count#26 ASC NULLS FIRST], [ca_state#19, cd_gender#22, cd_marital_status#23, cd_dep_count#24, cnt1#65, avg(cd_dep_count)#66, max(cd_dep_count)#67, sum(cd_dep_count)#68, cd_dep_employed_count#25, cnt2#69, avg(cd_dep_employed_count)#70, max(cd_dep_employed_count)#71, sum(cd_dep_employed_count)#72, cd_dep_college_count#26, cnt3#73, avg(cd_dep_college_count)#74, max(cd_dep_college_count)#75, sum(cd_dep_college_count)#76] +Arguments: 100, [ca_state#19 ASC NULLS FIRST, cd_gender#22 ASC NULLS FIRST, cd_marital_status#23 ASC NULLS FIRST, cd_dep_count#24 ASC NULLS FIRST, cd_dep_employed_count#25 ASC NULLS FIRST, cd_dep_college_count#26 ASC NULLS FIRST], [ca_state#19, cd_gender#22, cd_marital_status#23, cd_dep_count#24, cnt1#65, avg(cd_dep_count)#66, max(cd_dep_count)#67, sum(cd_dep_count)#68, cd_dep_employed_count#25, cnt2#69, avg(cd_dep_employed_count)#70, max(cd_dep_employed_count)#71, sum(cd_dep_employed_count)#72, cd_dep_college_count#26, cnt3#73, avg(cd_dep_college_count)#74, max(cd_dep_college_count)#75, sum(cd_dep_college_count)#76] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.sf100/explain.txt index f93a8221a587e..6302a3b221f62 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.sf100/explain.txt @@ -239,7 +239,7 @@ Input [6]: [gross_margin#22, i_category#13, i_class#12, lochierarchy#25, _w0#68, (41) TakeOrderedAndProject Input [5]: [gross_margin#22, i_category#13, i_class#12, lochierarchy#25, rank_within_parent#70] -Arguments: 100, 0, [lochierarchy#25 DESC NULLS LAST, CASE WHEN (lochierarchy#25 = 0) THEN i_category#13 END ASC NULLS FIRST, rank_within_parent#70 ASC NULLS FIRST], [gross_margin#22, i_category#13, i_class#12, lochierarchy#25, rank_within_parent#70] +Arguments: 100, [lochierarchy#25 DESC NULLS LAST, CASE WHEN (lochierarchy#25 = 0) THEN i_category#13 END ASC NULLS FIRST, rank_within_parent#70 ASC NULLS FIRST], [gross_margin#22, i_category#13, i_class#12, lochierarchy#25, rank_within_parent#70] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/explain.txt index d3e616eeda399..ecf954ef3c422 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/explain.txt @@ -239,7 +239,7 @@ Input [6]: [gross_margin#22, i_category#10, i_class#9, lochierarchy#25, _w0#68, (41) TakeOrderedAndProject Input [5]: [gross_margin#22, i_category#10, i_class#9, lochierarchy#25, rank_within_parent#70] -Arguments: 100, 0, [lochierarchy#25 DESC NULLS LAST, CASE WHEN (lochierarchy#25 = 0) THEN i_category#10 END ASC NULLS FIRST, rank_within_parent#70 ASC NULLS FIRST], [gross_margin#22, i_category#10, i_class#9, lochierarchy#25, rank_within_parent#70] +Arguments: 100, [lochierarchy#25 DESC NULLS LAST, CASE WHEN (lochierarchy#25 = 0) THEN i_category#10 END ASC NULLS FIRST, rank_within_parent#70 ASC NULLS FIRST], [gross_margin#22, i_category#10, i_class#9, lochierarchy#25, rank_within_parent#70] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.sf100/explain.txt index 6e9570ac21a48..4d6001619fe9b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.sf100/explain.txt @@ -277,7 +277,7 @@ Input [16]: [i_category#16, i_brand#15, s_store_name#10, s_company_name#11, d_ye (52) TakeOrderedAndProject Input [7]: [i_category#16, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#22, psum#49, nsum#50] -Arguments: 100, 0, [(sum_sales#22 - avg_monthly_sales#26) ASC NULLS FIRST, d_moy#8 ASC NULLS FIRST], [i_category#16, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#22, psum#49, nsum#50] +Arguments: 100, [(sum_sales#22 - avg_monthly_sales#26) ASC NULLS FIRST, d_moy#8 ASC NULLS FIRST], [i_category#16, d_year#7, d_moy#8, avg_monthly_sales#26, sum_sales#22, psum#49, nsum#50] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/explain.txt index 431765c39d499..712a9f8c7ea3b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/explain.txt @@ -242,7 +242,7 @@ Input [16]: [i_category#3, i_brand#2, s_store_name#14, s_company_name#15, d_year (45) TakeOrderedAndProject Input [7]: [i_category#3, d_year#11, d_moy#12, avg_monthly_sales#25, sum_sales#21, psum#47, nsum#48] -Arguments: 100, 0, [(sum_sales#21 - avg_monthly_sales#25) ASC NULLS FIRST, d_moy#12 ASC NULLS FIRST], [i_category#3, d_year#11, d_moy#12, avg_monthly_sales#25, sum_sales#21, psum#47, nsum#48] +Arguments: 100, [(sum_sales#21 - avg_monthly_sales#25) ASC NULLS FIRST, d_moy#12 ASC NULLS FIRST], [i_category#3, d_year#11, d_moy#12, avg_monthly_sales#25, sum_sales#21, psum#47, nsum#48] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.sf100/explain.txt index bddaba6017b02..c174a4f161788 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.sf100/explain.txt @@ -469,7 +469,7 @@ Results [5]: [channel#39, item#33, return_ratio#34, return_rank#37, currency_ran (86) TakeOrderedAndProject Input [5]: [channel#39, item#33, return_ratio#34, return_rank#37, currency_rank#38] -Arguments: 100, 0, [channel#39 ASC NULLS FIRST, return_rank#37 ASC NULLS FIRST, currency_rank#38 ASC NULLS FIRST, item#33 ASC NULLS FIRST], [channel#39, item#33, return_ratio#34, return_rank#37, currency_rank#38] +Arguments: 100, [channel#39 ASC NULLS FIRST, return_rank#37 ASC NULLS FIRST, currency_rank#38 ASC NULLS FIRST, item#33 ASC NULLS FIRST], [channel#39, item#33, return_ratio#34, return_rank#37, currency_rank#38] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/explain.txt index ae71bfbac6de8..ddb9caf89cbde 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/explain.txt @@ -424,7 +424,7 @@ Results [5]: [channel#38, item#32, return_ratio#33, return_rank#36, currency_ran (77) TakeOrderedAndProject Input [5]: [channel#38, item#32, return_ratio#33, return_rank#36, currency_rank#37] -Arguments: 100, 0, [channel#38 ASC NULLS FIRST, return_rank#36 ASC NULLS FIRST, currency_rank#37 ASC NULLS FIRST, item#32 ASC NULLS FIRST], [channel#38, item#32, return_ratio#33, return_rank#36, currency_rank#37] +Arguments: 100, [channel#38 ASC NULLS FIRST, return_rank#36 ASC NULLS FIRST, currency_rank#37 ASC NULLS FIRST, item#32 ASC NULLS FIRST], [channel#38, item#32, return_ratio#33, return_rank#36, currency_rank#37] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.sf100/explain.txt index 7c0c4f63bd4fb..740ea0f9ebbd2 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.sf100/explain.txt @@ -386,7 +386,7 @@ Condition : ((isnotnull(web_cumulative#72) AND isnotnull(store_cumulative#73)) A (70) TakeOrderedAndProject Input [6]: [item_sk#56, d_date#57, web_sales#58, store_sales#59, web_cumulative#72, store_cumulative#73] -Arguments: 100, 0, [item_sk#56 ASC NULLS FIRST, d_date#57 ASC NULLS FIRST], [item_sk#56, d_date#57, web_sales#58, store_sales#59, web_cumulative#72, store_cumulative#73] +Arguments: 100, [item_sk#56 ASC NULLS FIRST, d_date#57 ASC NULLS FIRST], [item_sk#56, d_date#57, web_sales#58, store_sales#59, web_cumulative#72, store_cumulative#73] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/explain.txt index 839277f4bf3d1..cf86cd670456f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/explain.txt @@ -371,7 +371,7 @@ Condition : ((isnotnull(web_cumulative#73) AND isnotnull(store_cumulative#74)) A (67) TakeOrderedAndProject Input [6]: [item_sk#56, d_date#57, web_sales#58, store_sales#59, web_cumulative#73, store_cumulative#74] -Arguments: 100, 0, [item_sk#56 ASC NULLS FIRST, d_date#57 ASC NULLS FIRST], [item_sk#56, d_date#57, web_sales#58, store_sales#59, web_cumulative#73, store_cumulative#74] +Arguments: 100, [item_sk#56 ASC NULLS FIRST, d_date#57 ASC NULLS FIRST], [item_sk#56, d_date#57, web_sales#58, store_sales#59, web_cumulative#73, store_cumulative#74] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.sf100/explain.txt index 1d27ee0e8f6cd..e07d4460643c3 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.sf100/explain.txt @@ -277,7 +277,7 @@ Input [14]: [i_category#15, i_brand#14, cc_name#10, d_year#7, d_moy#8, sum_sales (52) TakeOrderedAndProject Input [8]: [i_category#15, i_brand#14, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#21, psum#46, nsum#47] -Arguments: 100, 0, [(sum_sales#21 - avg_monthly_sales#25) ASC NULLS FIRST, d_year#7 ASC NULLS FIRST], [i_category#15, i_brand#14, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#21, psum#46, nsum#47] +Arguments: 100, [(sum_sales#21 - avg_monthly_sales#25) ASC NULLS FIRST, d_year#7 ASC NULLS FIRST], [i_category#15, i_brand#14, d_year#7, d_moy#8, avg_monthly_sales#25, sum_sales#21, psum#46, nsum#47] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/explain.txt index dfcb28de5e336..1cd7e4928cc0a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/explain.txt @@ -242,7 +242,7 @@ Input [14]: [i_category#3, i_brand#2, cc_name#14, d_year#11, d_moy#12, sum_sales (45) TakeOrderedAndProject Input [8]: [i_category#3, i_brand#2, d_year#11, d_moy#12, avg_monthly_sales#24, sum_sales#20, psum#44, nsum#45] -Arguments: 100, 0, [(sum_sales#20 - avg_monthly_sales#24) ASC NULLS FIRST, d_year#11 ASC NULLS FIRST], [i_category#3, i_brand#2, d_year#11, d_moy#12, avg_monthly_sales#24, sum_sales#20, psum#44, nsum#45] +Arguments: 100, [(sum_sales#20 - avg_monthly_sales#24) ASC NULLS FIRST, d_year#11 ASC NULLS FIRST], [i_category#3, i_brand#2, d_year#11, d_moy#12, avg_monthly_sales#24, sum_sales#20, psum#44, nsum#45] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.sf100/explain.txt index 64ad1725891b9..f6795596efb99 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.sf100/explain.txt @@ -514,7 +514,7 @@ Results [5]: [channel#39, id#40, sales#151, returns#152, profit#153] (90) TakeOrderedAndProject Input [5]: [channel#39, id#40, sales#151, returns#152, profit#153] -Arguments: 100, 0, [channel#39 ASC NULLS FIRST, id#40 ASC NULLS FIRST], [channel#39, id#40, sales#151, returns#152, profit#153] +Arguments: 100, [channel#39 ASC NULLS FIRST, id#40 ASC NULLS FIRST], [channel#39, id#40, sales#151, returns#152, profit#153] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/explain.txt index 230b85eb47e03..db39a5768b988 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/explain.txt @@ -499,7 +499,7 @@ Results [5]: [channel#39, id#40, sales#150, returns#151, profit#152] (87) TakeOrderedAndProject Input [5]: [channel#39, id#40, sales#150, returns#151, profit#152] -Arguments: 100, 0, [channel#39 ASC NULLS FIRST, id#40 ASC NULLS FIRST], [channel#39, id#40, sales#150, returns#151, profit#152] +Arguments: 100, [channel#39 ASC NULLS FIRST, id#40 ASC NULLS FIRST], [channel#39, id#40, sales#150, returns#151, profit#152] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.sf100/explain.txt index 648dcf6b9ac8f..25c78871aedf5 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.sf100/explain.txt @@ -251,7 +251,7 @@ Condition : (cnt#33 >= 10) (45) TakeOrderedAndProject Input [3]: [state#32, cnt#33, ca_state#22] -Arguments: 100, 0, [cnt#33 ASC NULLS FIRST, ca_state#22 ASC NULLS FIRST], [state#32, cnt#33] +Arguments: 100, [cnt#33 ASC NULLS FIRST, ca_state#22 ASC NULLS FIRST], [state#32, cnt#33] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/explain.txt index 9be5e8772388e..e9b40299e3676 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/explain.txt @@ -221,7 +221,7 @@ Condition : (cnt#31 >= 10) (39) TakeOrderedAndProject Input [3]: [state#30, cnt#31, ca_state#2] -Arguments: 100, 0, [cnt#31 ASC NULLS FIRST, ca_state#2 ASC NULLS FIRST], [state#30, cnt#31] +Arguments: 100, [cnt#31 ASC NULLS FIRST, ca_state#2 ASC NULLS FIRST], [state#30, cnt#31] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.sf100/explain.txt index 2db495350c5d4..cbedda1a7bee6 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.sf100/explain.txt @@ -426,7 +426,7 @@ Condition : (rk#138 <= 100) (70) TakeOrderedAndProject Input [10]: [i_category#18, i_class#17, i_brand#16, i_product_name#19, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#27, rk#138] -Arguments: 100, 0, [i_category#18 ASC NULLS FIRST, i_class#17 ASC NULLS FIRST, i_brand#16 ASC NULLS FIRST, i_product_name#19 ASC NULLS FIRST, d_year#8 ASC NULLS FIRST, d_qoy#10 ASC NULLS FIRST, d_moy#9 ASC NULLS FIRST, s_store_id#12 ASC NULLS FIRST, sumsales#27 ASC NULLS FIRST, rk#138 ASC NULLS FIRST], [i_category#18, i_class#17, i_brand#16, i_product_name#19, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#27, rk#138] +Arguments: 100, [i_category#18 ASC NULLS FIRST, i_class#17 ASC NULLS FIRST, i_brand#16 ASC NULLS FIRST, i_product_name#19 ASC NULLS FIRST, d_year#8 ASC NULLS FIRST, d_qoy#10 ASC NULLS FIRST, d_moy#9 ASC NULLS FIRST, s_store_id#12 ASC NULLS FIRST, sumsales#27 ASC NULLS FIRST, rk#138 ASC NULLS FIRST], [i_category#18, i_class#17, i_brand#16, i_product_name#19, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#27, rk#138] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/explain.txt index 65e590216b11f..fd3033f54554e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/explain.txt @@ -411,7 +411,7 @@ Condition : (rk#137 <= 100) (67) TakeOrderedAndProject Input [10]: [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#26, rk#137] -Arguments: 100, 0, [i_category#17 ASC NULLS FIRST, i_class#16 ASC NULLS FIRST, i_brand#15 ASC NULLS FIRST, i_product_name#18 ASC NULLS FIRST, d_year#8 ASC NULLS FIRST, d_qoy#10 ASC NULLS FIRST, d_moy#9 ASC NULLS FIRST, s_store_id#12 ASC NULLS FIRST, sumsales#26 ASC NULLS FIRST, rk#137 ASC NULLS FIRST], [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#26, rk#137] +Arguments: 100, [i_category#17 ASC NULLS FIRST, i_class#16 ASC NULLS FIRST, i_brand#15 ASC NULLS FIRST, i_product_name#18 ASC NULLS FIRST, d_year#8 ASC NULLS FIRST, d_qoy#10 ASC NULLS FIRST, d_moy#9 ASC NULLS FIRST, s_store_id#12 ASC NULLS FIRST, sumsales#26 ASC NULLS FIRST, rk#137 ASC NULLS FIRST], [i_category#17, i_class#16, i_brand#15, i_product_name#18, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#26, rk#137] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.sf100/explain.txt index 0e8c3ac9f8ae1..015dc51d2aec6 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.sf100/explain.txt @@ -324,7 +324,7 @@ Input [6]: [total_sum#30, s_state#8, s_county#7, lochierarchy#33, _w0#61, rank_w (56) TakeOrderedAndProject Input [5]: [total_sum#30, s_state#8, s_county#7, lochierarchy#33, rank_within_parent#63] -Arguments: 100, 0, [lochierarchy#33 DESC NULLS LAST, CASE WHEN (lochierarchy#33 = 0) THEN s_state#8 END ASC NULLS FIRST, rank_within_parent#63 ASC NULLS FIRST], [total_sum#30, s_state#8, s_county#7, lochierarchy#33, rank_within_parent#63] +Arguments: 100, [lochierarchy#33 DESC NULLS LAST, CASE WHEN (lochierarchy#33 = 0) THEN s_state#8 END ASC NULLS FIRST, rank_within_parent#63 ASC NULLS FIRST], [total_sum#30, s_state#8, s_county#7, lochierarchy#33, rank_within_parent#63] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/explain.txt index 5db98387feb61..a67d7e109e7fe 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/explain.txt @@ -324,7 +324,7 @@ Input [6]: [total_sum#30, s_state#8, s_county#7, lochierarchy#33, _w0#61, rank_w (56) TakeOrderedAndProject Input [5]: [total_sum#30, s_state#8, s_county#7, lochierarchy#33, rank_within_parent#63] -Arguments: 100, 0, [lochierarchy#33 DESC NULLS LAST, CASE WHEN (lochierarchy#33 = 0) THEN s_state#8 END ASC NULLS FIRST, rank_within_parent#63 ASC NULLS FIRST], [total_sum#30, s_state#8, s_county#7, lochierarchy#33, rank_within_parent#63] +Arguments: 100, [lochierarchy#33 DESC NULLS LAST, CASE WHEN (lochierarchy#33 = 0) THEN s_state#8 END ASC NULLS FIRST, rank_within_parent#63 ASC NULLS FIRST], [total_sum#30, s_state#8, s_county#7, lochierarchy#33, rank_within_parent#63] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/explain.txt index 4169bbb51d9d6..e5e42f2be1366 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/explain.txt @@ -383,7 +383,7 @@ Results [6]: [i_item_desc#21, w_warehouse_name#33, d_week_seq#25, count(1)#46 AS (70) TakeOrderedAndProject Input [6]: [i_item_desc#21, w_warehouse_name#33, d_week_seq#25, no_promo#47, promo#48, total_cnt#49] -Arguments: 100, 0, [total_cnt#49 DESC NULLS LAST, i_item_desc#21 ASC NULLS FIRST, w_warehouse_name#33 ASC NULLS FIRST, d_week_seq#25 ASC NULLS FIRST], [i_item_desc#21, w_warehouse_name#33, d_week_seq#25, no_promo#47, promo#48, total_cnt#49] +Arguments: 100, [total_cnt#49 DESC NULLS LAST, i_item_desc#21 ASC NULLS FIRST, w_warehouse_name#33 ASC NULLS FIRST, d_week_seq#25 ASC NULLS FIRST], [i_item_desc#21, w_warehouse_name#33, d_week_seq#25, no_promo#47, promo#48, total_cnt#49] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/explain.txt index ae21c596d5bd0..33ef0ab30ebc9 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/explain.txt @@ -386,7 +386,7 @@ Results [6]: [i_item_desc#19, w_warehouse_name#16, d_week_seq#29, count(1)#46 AS (70) TakeOrderedAndProject Input [6]: [i_item_desc#19, w_warehouse_name#16, d_week_seq#29, no_promo#47, promo#48, total_cnt#49] -Arguments: 100, 0, [total_cnt#49 DESC NULLS LAST, i_item_desc#19 ASC NULLS FIRST, w_warehouse_name#16 ASC NULLS FIRST, d_week_seq#29 ASC NULLS FIRST], [i_item_desc#19, w_warehouse_name#16, d_week_seq#29, no_promo#47, promo#48, total_cnt#49] +Arguments: 100, [total_cnt#49 DESC NULLS LAST, i_item_desc#19 ASC NULLS FIRST, w_warehouse_name#16 ASC NULLS FIRST, d_week_seq#29 ASC NULLS FIRST], [i_item_desc#19, w_warehouse_name#16, d_week_seq#29, no_promo#47, promo#48, total_cnt#49] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.sf100/explain.txt index 1ad1831fcd87d..3b21d6fd03243 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.sf100/explain.txt @@ -441,7 +441,7 @@ Input [9]: [customer_id#17, year_total#18, customer_id#34, customer_first_name#3 (80) TakeOrderedAndProject Input [3]: [customer_id#34, customer_first_name#35, customer_last_name#36] -Arguments: 100, 0, [customer_first_name#35 ASC NULLS FIRST, customer_id#34 ASC NULLS FIRST, customer_last_name#36 ASC NULLS FIRST], [customer_id#34, customer_first_name#35, customer_last_name#36] +Arguments: 100, [customer_first_name#35 ASC NULLS FIRST, customer_id#34 ASC NULLS FIRST, customer_last_name#36 ASC NULLS FIRST], [customer_id#34, customer_first_name#35, customer_last_name#36] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/explain.txt index 5ff8bb0039112..a3a369626beb5 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/explain.txt @@ -410,7 +410,7 @@ Input [9]: [customer_id#16, year_total#17, customer_id#32, customer_first_name#3 (72) TakeOrderedAndProject Input [3]: [customer_id#32, customer_first_name#33, customer_last_name#34] -Arguments: 100, 0, [customer_first_name#33 ASC NULLS FIRST, customer_id#32 ASC NULLS FIRST, customer_last_name#34 ASC NULLS FIRST], [customer_id#32, customer_first_name#33, customer_last_name#34] +Arguments: 100, [customer_first_name#33 ASC NULLS FIRST, customer_id#32 ASC NULLS FIRST, customer_last_name#34 ASC NULLS FIRST], [customer_id#32, customer_first_name#33, customer_last_name#34] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.sf100/explain.txt index e32c4f29ee131..7e213628bfc18 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.sf100/explain.txt @@ -695,7 +695,7 @@ Input [14]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact (129) TakeOrderedAndProject Input [10]: [prev_year#137, year#138, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#139, curr_yr_cnt#140, sales_cnt_diff#141, sales_amt_diff#142] -Arguments: 100, 0, [sales_cnt_diff#141 ASC NULLS FIRST, sales_amt_diff#142 ASC NULLS FIRST], [prev_year#137, year#138, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#139, curr_yr_cnt#140, sales_cnt_diff#141, sales_amt_diff#142] +Arguments: 100, [sales_cnt_diff#141 ASC NULLS FIRST, sales_amt_diff#142 ASC NULLS FIRST], [prev_year#137, year#138, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#139, curr_yr_cnt#140, sales_cnt_diff#141, sales_amt_diff#142] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/explain.txt index e32c4f29ee131..7e213628bfc18 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/explain.txt @@ -695,7 +695,7 @@ Input [14]: [d_year#15, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact (129) TakeOrderedAndProject Input [10]: [prev_year#137, year#138, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#139, curr_yr_cnt#140, sales_cnt_diff#141, sales_amt_diff#142] -Arguments: 100, 0, [sales_cnt_diff#141 ASC NULLS FIRST, sales_amt_diff#142 ASC NULLS FIRST], [prev_year#137, year#138, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#139, curr_yr_cnt#140, sales_cnt_diff#141, sales_amt_diff#142] +Arguments: 100, [sales_cnt_diff#141 ASC NULLS FIRST, sales_amt_diff#142 ASC NULLS FIRST], [prev_year#137, year#138, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#139, curr_yr_cnt#140, sales_cnt_diff#141, sales_amt_diff#142] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.sf100/explain.txt index 461007be5b3a9..27945848a133a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.sf100/explain.txt @@ -572,7 +572,7 @@ Results [5]: [channel#34, id#35, sales#121, returns#122, profit#123] (98) TakeOrderedAndProject Input [5]: [channel#34, id#35, sales#121, returns#122, profit#123] -Arguments: 100, 0, [channel#34 ASC NULLS FIRST, id#35 ASC NULLS FIRST], [channel#34, id#35, sales#121, returns#122, profit#123] +Arguments: 100, [channel#34 ASC NULLS FIRST, id#35 ASC NULLS FIRST], [channel#34, id#35, sales#121, returns#122, profit#123] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/explain.txt index 65661a80f9f84..74f61a018b497 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/explain.txt @@ -572,7 +572,7 @@ Results [5]: [channel#34, id#35, sales#121, returns#122, profit#123] (98) TakeOrderedAndProject Input [5]: [channel#34, id#35, sales#121, returns#122, profit#123] -Arguments: 100, 0, [channel#34 ASC NULLS FIRST, id#35 ASC NULLS FIRST], [channel#34, id#35, sales#121, returns#122, profit#123] +Arguments: 100, [channel#34 ASC NULLS FIRST, id#35 ASC NULLS FIRST], [channel#34, id#35, sales#121, returns#122, profit#123] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.sf100/explain.txt index bf191ba233304..c54f269af69b6 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.sf100/explain.txt @@ -387,7 +387,7 @@ Input [15]: [ss_sold_year#26, ss_item_sk#1, ss_customer_sk#2, ss_qty#27, ss_wc#2 (70) TakeOrderedAndProject Input [13]: [ratio#88, store_qty#89, store_wholesale_cost#90, store_sales_price#91, other_chan_qty#92, other_chan_wholesale_cost#93, other_chan_sales_price#94, ss_sold_year#26, ss_item_sk#1, ss_customer_sk#2, ss_qty#27, ss_wc#28, ss_sp#29] -Arguments: 100, 0, [ss_sold_year#26 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST, ss_qty#27 DESC NULLS LAST, ss_wc#28 DESC NULLS LAST, ss_sp#29 DESC NULLS LAST, other_chan_qty#92 ASC NULLS FIRST, other_chan_wholesale_cost#93 ASC NULLS FIRST, other_chan_sales_price#94 ASC NULLS FIRST, ratio#88 ASC NULLS FIRST], [ratio#88, store_qty#89, store_wholesale_cost#90, store_sales_price#91, other_chan_qty#92, other_chan_wholesale_cost#93, other_chan_sales_price#94] +Arguments: 100, [ss_sold_year#26 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST, ss_qty#27 DESC NULLS LAST, ss_wc#28 DESC NULLS LAST, ss_sp#29 DESC NULLS LAST, other_chan_qty#92 ASC NULLS FIRST, other_chan_wholesale_cost#93 ASC NULLS FIRST, other_chan_sales_price#94 ASC NULLS FIRST, ratio#88 ASC NULLS FIRST], [ratio#88, store_qty#89, store_wholesale_cost#90, store_sales_price#91, other_chan_qty#92, other_chan_wholesale_cost#93, other_chan_sales_price#94] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/explain.txt index bf191ba233304..c54f269af69b6 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/explain.txt @@ -387,7 +387,7 @@ Input [15]: [ss_sold_year#26, ss_item_sk#1, ss_customer_sk#2, ss_qty#27, ss_wc#2 (70) TakeOrderedAndProject Input [13]: [ratio#88, store_qty#89, store_wholesale_cost#90, store_sales_price#91, other_chan_qty#92, other_chan_wholesale_cost#93, other_chan_sales_price#94, ss_sold_year#26, ss_item_sk#1, ss_customer_sk#2, ss_qty#27, ss_wc#28, ss_sp#29] -Arguments: 100, 0, [ss_sold_year#26 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST, ss_qty#27 DESC NULLS LAST, ss_wc#28 DESC NULLS LAST, ss_sp#29 DESC NULLS LAST, other_chan_qty#92 ASC NULLS FIRST, other_chan_wholesale_cost#93 ASC NULLS FIRST, other_chan_sales_price#94 ASC NULLS FIRST, ratio#88 ASC NULLS FIRST], [ratio#88, store_qty#89, store_wholesale_cost#90, store_sales_price#91, other_chan_qty#92, other_chan_wholesale_cost#93, other_chan_sales_price#94] +Arguments: 100, [ss_sold_year#26 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST, ss_qty#27 DESC NULLS LAST, ss_wc#28 DESC NULLS LAST, ss_sp#29 DESC NULLS LAST, other_chan_qty#92 ASC NULLS FIRST, other_chan_wholesale_cost#93 ASC NULLS FIRST, other_chan_sales_price#94 ASC NULLS FIRST, ratio#88 ASC NULLS FIRST], [ratio#88, store_qty#89, store_wholesale_cost#90, store_sales_price#91, other_chan_qty#92, other_chan_wholesale_cost#93, other_chan_sales_price#94] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.sf100/explain.txt index 30fdcd756313b..ec5fc20a86a73 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.sf100/explain.txt @@ -674,7 +674,7 @@ Results [5]: [channel#40, id#41, sales#139, returns#140, profit#141] (120) TakeOrderedAndProject Input [5]: [channel#40, id#41, sales#139, returns#140, profit#141] -Arguments: 100, 0, [channel#40 ASC NULLS FIRST, id#41 ASC NULLS FIRST], [channel#40, id#41, sales#139, returns#140, profit#141] +Arguments: 100, [channel#40 ASC NULLS FIRST, id#41 ASC NULLS FIRST], [channel#40, id#41, sales#139, returns#140, profit#141] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/explain.txt index 43a97b4cb7ff5..3ce8c5db2b207 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/explain.txt @@ -674,7 +674,7 @@ Results [5]: [channel#40, id#41, sales#139, returns#140, profit#141] (120) TakeOrderedAndProject Input [5]: [channel#40, id#41, sales#139, returns#140, profit#141] -Arguments: 100, 0, [channel#40 ASC NULLS FIRST, id#41 ASC NULLS FIRST], [channel#40, id#41, sales#139, returns#140, profit#141] +Arguments: 100, [channel#40 ASC NULLS FIRST, id#41 ASC NULLS FIRST], [channel#40, id#41, sales#139, returns#140, profit#141] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.sf100/explain.txt index 19be39c47ba65..1dd3dc76a8f56 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.sf100/explain.txt @@ -201,7 +201,7 @@ Input [6]: [total_sum#14, i_category#8, i_class#7, lochierarchy#17, _w0#45, rank (34) TakeOrderedAndProject Input [5]: [total_sum#14, i_category#8, i_class#7, lochierarchy#17, rank_within_parent#47] -Arguments: 100, 0, [lochierarchy#17 DESC NULLS LAST, CASE WHEN (lochierarchy#17 = 0) THEN i_category#8 END ASC NULLS FIRST, rank_within_parent#47 ASC NULLS FIRST], [total_sum#14, i_category#8, i_class#7, lochierarchy#17, rank_within_parent#47] +Arguments: 100, [lochierarchy#17 DESC NULLS LAST, CASE WHEN (lochierarchy#17 = 0) THEN i_category#8 END ASC NULLS FIRST, rank_within_parent#47 ASC NULLS FIRST], [total_sum#14, i_category#8, i_class#7, lochierarchy#17, rank_within_parent#47] ===== Subqueries ===== diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/explain.txt index 19be39c47ba65..1dd3dc76a8f56 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/explain.txt @@ -201,7 +201,7 @@ Input [6]: [total_sum#14, i_category#8, i_class#7, lochierarchy#17, _w0#45, rank (34) TakeOrderedAndProject Input [5]: [total_sum#14, i_category#8, i_class#7, lochierarchy#17, rank_within_parent#47] -Arguments: 100, 0, [lochierarchy#17 DESC NULLS LAST, CASE WHEN (lochierarchy#17 = 0) THEN i_category#8 END ASC NULLS FIRST, rank_within_parent#47 ASC NULLS FIRST], [total_sum#14, i_category#8, i_class#7, lochierarchy#17, rank_within_parent#47] +Arguments: 100, [lochierarchy#17 DESC NULLS LAST, CASE WHEN (lochierarchy#17 = 0) THEN i_category#8 END ASC NULLS FIRST, rank_within_parent#47 ASC NULLS FIRST], [total_sum#14, i_category#8, i_class#7, lochierarchy#17, rank_within_parent#47] ===== Subqueries ===== diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala index 2fd59932d230e..2e2fdd17207bb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala @@ -603,6 +603,30 @@ class DataFrameSuite extends QueryTest ) } + test("offset") { + checkAnswer( + testData.offset(90), + testData.collect().drop(90).toSeq) + + checkAnswer( + arrayData.toDF().offset(99), + arrayData.collect().drop(99).map(r => Row.fromSeq(r.productIterator.toSeq))) + + checkAnswer( + mapData.toDF().offset(99), + mapData.collect().drop(99).map(r => Row.fromSeq(r.productIterator.toSeq))) + } + + test("limit with offset") { + checkAnswer( + testData.limit(10).offset(5), + testData.take(10).drop(5).toSeq) + + checkAnswer( + testData.offset(5).limit(10), + testData.take(15).drop(5).toSeq) + } + test("udf") { val foo = udf((a: Int, b: String) => a.toString + b) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/TakeOrderedAndProjectSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/TakeOrderedAndProjectSuite.scala index 0f6c808b16708..6ec5c6287eed1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/TakeOrderedAndProjectSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/TakeOrderedAndProjectSuite.scala @@ -66,7 +66,7 @@ class TakeOrderedAndProjectSuite extends SparkPlanTest with SharedSparkSession { checkThatPlansAgree( generateRandomInputData(n, m), input => - noOpFilter(TakeOrderedAndProjectExec(limit, 0, sortOrder, input.output, input)), + noOpFilter(TakeOrderedAndProjectExec(limit, sortOrder, input.output, input)), input => GlobalLimitExec(limit, LocalLimitExec(limit, @@ -83,7 +83,7 @@ class TakeOrderedAndProjectSuite extends SparkPlanTest with SharedSparkSession { generateRandomInputData(n, m), input => noOpFilter( - TakeOrderedAndProjectExec(limit, 0, sortOrder, Seq(input.output.last), input)), + TakeOrderedAndProjectExec(limit, sortOrder, Seq(input.output.last), input)), input => GlobalLimitExec(limit, LocalLimitExec(limit, diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2StrategySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2StrategySuite.scala index 6296da47cca51..1a5a382afdc6b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2StrategySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2StrategySuite.scala @@ -37,7 +37,7 @@ class DataSourceV2StrategySuite extends PlanTest with SharedSparkSession { */ def testTranslateFilter(catalystFilter: Expression, result: Option[Predicate]): Unit = { assertResult(result) { - DataSourceV2Strategy.translateFilterV2(catalystFilter, true) + DataSourceV2Strategy.translateFilterV2(catalystFilter) } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCV2Suite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCV2Suite.scala index eb8ca9ea042a0..2a5f88234513a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCV2Suite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCV2Suite.scala @@ -23,11 +23,11 @@ import java.util.Properties import org.apache.spark.{SparkConf, SparkException} import org.apache.spark.sql.{AnalysisException, DataFrame, ExplainSuiteHelper, QueryTest, Row} import org.apache.spark.sql.catalyst.analysis.CannotReplaceMissingTableException -import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, GlobalLimit, LocalLimit, Sort} +import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, GlobalLimit, LocalLimit, Offset, Sort} import org.apache.spark.sql.connector.IntegralAverage import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2ScanRelation, V1ScanWrapper} import org.apache.spark.sql.execution.datasources.v2.jdbc.JDBCTableCatalog -import org.apache.spark.sql.functions.{abs, avg, coalesce, count, count_distinct, exp, lit, log => ln, not, pow, sqrt, sum, udf, when} +import org.apache.spark.sql.functions.{abs, acos, asin, atan, atan2, avg, ceil, coalesce, cos, cosh, count, count_distinct, degrees, exp, floor, lit, log => logarithm, log10, not, pow, radians, round, signum, sin, sinh, sqrt, sum, tan, tanh, udf, when} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.util.Utils @@ -45,6 +45,7 @@ class JDBCV2Suite extends QueryTest with SharedSparkSession with ExplainSuiteHel .set("spark.sql.catalog.h2.driver", "org.h2.Driver") .set("spark.sql.catalog.h2.pushDownAggregate", "true") .set("spark.sql.catalog.h2.pushDownLimit", "true") + .set("spark.sql.catalog.h2.pushDownOffset", "true") private def withConnection[T](f: Connection => T): T = { val conn = DriverManager.getConnection(url, new Properties()) @@ -82,9 +83,10 @@ class JDBCV2Suite extends QueryTest with SharedSparkSession with ExplainSuiteHel conn.prepareStatement( "INSERT INTO \"test\".\"employee\" VALUES (6, 'jen', 12000, 1200, true)").executeUpdate() conn.prepareStatement( - "CREATE TABLE \"test\".\"dept\" (\"dept_id\" INTEGER NOT NULL)").executeUpdate() - conn.prepareStatement("INSERT INTO \"test\".\"dept\" VALUES (1)").executeUpdate() - conn.prepareStatement("INSERT INTO \"test\".\"dept\" VALUES (2)").executeUpdate() + "CREATE TABLE \"test\".\"dept\" (\"dept_id\" INTEGER NOT NULL, \"dept_id1\" INTEGER)") + .executeUpdate() + conn.prepareStatement("INSERT INTO \"test\".\"dept\" VALUES (1, 1)").executeUpdate() + conn.prepareStatement("INSERT INTO \"test\".\"dept\" VALUES (2, 1)").executeUpdate() // scalastyle:off conn.prepareStatement( @@ -120,10 +122,10 @@ class JDBCV2Suite extends QueryTest with SharedSparkSession with ExplainSuiteHel checkAnswer(sql("SELECT name, id FROM h2.test.people"), Seq(Row("fred", 1), Row("mary", 2))) } - private def checkPushedInfo(df: DataFrame, expectedPlanFragment: String): Unit = { + private def checkPushedInfo(df: DataFrame, expectedPlanFragment: String*): Unit = { df.queryExecution.optimizedPlan.collect { case _: DataSourceV2ScanRelation => - checkKeywordsExistsInExplain(df, expectedPlanFragment) + checkKeywordsExistsInExplain(df, expectedPlanFragment: _*) } } @@ -206,6 +208,355 @@ class JDBCV2Suite extends QueryTest with SharedSparkSession with ExplainSuiteHel checkAnswer(df5, Seq(Row(10000.00, 1000.0, "amy"))) } + private def checkOffsetRemoved(df: DataFrame, removed: Boolean = true): Unit = { + val offsets = df.queryExecution.optimizedPlan.collect { + case offset: Offset => offset + } + if (removed) { + assert(offsets.isEmpty) + } else { + assert(offsets.nonEmpty) + } + } + + test("simple scan with OFFSET") { + val df1 = spark.read + .table("h2.test.employee") + .where($"dept" === 1) + .offset(1) + checkOffsetRemoved(df1) + checkPushedInfo(df1, + "PushedFilters: [DEPT IS NOT NULL, DEPT = 1], PushedOffset: OFFSET 1,") + checkAnswer(df1, Seq(Row(1, "cathy", 9000.00, 1200.0, false))) + + val df2 = spark.read + .option("pushDownOffset", "false") + .table("h2.test.employee") + .where($"dept" === 1) + .offset(1) + checkOffsetRemoved(df2, false) + checkPushedInfo(df2, + "PushedFilters: [DEPT IS NOT NULL, DEPT = 1], ReadSchema:") + checkAnswer(df2, Seq(Row(1, "cathy", 9000.00, 1200.0, false))) + + val df3 = spark.read + .table("h2.test.employee") + .where($"dept" === 1) + .sort($"salary") + .offset(1) + checkOffsetRemoved(df3, false) + checkPushedInfo(df3, + "PushedFilters: [DEPT IS NOT NULL, DEPT = 1], ReadSchema:") + checkAnswer(df3, Seq(Row(1, "amy", 10000.00, 1000.0, true))) + + val df4 = spark.read + .option("partitionColumn", "dept") + .option("lowerBound", "0") + .option("upperBound", "2") + .option("numPartitions", "2") + .table("h2.test.employee") + .filter($"dept" > 1) + .offset(1) + checkOffsetRemoved(df4, false) + checkPushedInfo(df4, "PushedFilters: [DEPT IS NOT NULL, DEPT > 1], ReadSchema:") + checkAnswer(df4, Seq(Row(2, "david", 10000, 1300, true), Row(6, "jen", 12000, 1200, true))) + + val df5 = spark.read + .table("h2.test.employee") + .groupBy("DEPT").sum("SALARY") + .offset(1) + checkOffsetRemoved(df5, false) + checkPushedInfo(df5, + "PushedAggregates: [SUM(SALARY)], PushedFilters: [], PushedGroupByExpressions: [DEPT], ") + checkAnswer(df5, Seq(Row(2, 22000.00), Row(6, 12000.00))) + + val name = udf { (x: String) => x.matches("cat|dav|amy") } + val sub = udf { (x: String) => x.substring(0, 3) } + val df6 = spark.read + .table("h2.test.employee") + .select($"SALARY", $"BONUS", sub($"NAME").as("shortName")) + .filter(name($"shortName")) + .offset(1) + checkOffsetRemoved(df6, false) + // OFFSET is pushed down only if all the filters are pushed down + checkPushedInfo(df6, "PushedFilters: [], ") + checkAnswer(df6, Seq(Row(10000.00, 1300.0, "dav"), Row(9000.00, 1200.0, "cat"))) + } + + test("simple scan with LIMIT and OFFSET") { + val df1 = spark.read + .table("h2.test.employee") + .where($"dept" === 1) + .limit(2) + .offset(1) + checkLimitRemoved(df1) + checkOffsetRemoved(df1) + checkPushedInfo(df1, + "PushedFilters: [DEPT IS NOT NULL, DEPT = 1], PushedLimit: LIMIT 2, PushedOffset: OFFSET 1,") + checkAnswer(df1, Seq(Row(1, "cathy", 9000.00, 1200.0, false))) + + val df2 = spark.read + .option("pushDownLimit", "false") + .table("h2.test.employee") + .where($"dept" === 1) + .limit(2) + .offset(1) + checkLimitRemoved(df2, false) + checkOffsetRemoved(df2, false) + checkPushedInfo(df2, + "PushedFilters: [DEPT IS NOT NULL, DEPT = 1], ReadSchema:") + checkAnswer(df2, Seq(Row(1, "cathy", 9000.00, 1200.0, false))) + + val df3 = spark.read + .option("pushDownOffset", "false") + .table("h2.test.employee") + .where($"dept" === 1) + .limit(2) + .offset(1) + checkLimitRemoved(df3) + checkOffsetRemoved(df3, false) + checkPushedInfo(df3, + "PushedFilters: [DEPT IS NOT NULL, DEPT = 1], PushedLimit: LIMIT 2, ReadSchema:") + checkAnswer(df3, Seq(Row(1, "cathy", 9000.00, 1200.0, false))) + + val df4 = spark.read + .option("pushDownLimit", "false") + .option("pushDownOffset", "false") + .table("h2.test.employee") + .where($"dept" === 1) + .limit(2) + .offset(1) + checkLimitRemoved(df4, false) + checkOffsetRemoved(df4, false) + checkPushedInfo(df4, + "PushedFilters: [DEPT IS NOT NULL, DEPT = 1], ReadSchema:") + checkAnswer(df4, Seq(Row(1, "cathy", 9000.00, 1200.0, false))) + + val df5 = spark.read + .table("h2.test.employee") + .where($"dept" === 1) + .sort($"salary") + .limit(2) + .offset(1) + checkLimitRemoved(df5) + checkOffsetRemoved(df5) + checkPushedInfo(df5, "PushedFilters: [DEPT IS NOT NULL, DEPT = 1], " + + "PushedOffset: OFFSET 1, PushedTopN: ORDER BY [SALARY ASC NULLS FIRST] LIMIT 2, ReadSchema:") + checkAnswer(df5, Seq(Row(1, "amy", 10000.00, 1000.0, true))) + + val df6 = spark.read + .option("pushDownLimit", "false") + .table("h2.test.employee") + .where($"dept" === 1) + .sort($"salary") + .limit(2) + .offset(1) + checkLimitRemoved(df6, false) + checkOffsetRemoved(df6, false) + checkPushedInfo(df6, "PushedFilters: [DEPT IS NOT NULL, DEPT = 1], ReadSchema:") + checkAnswer(df6, Seq(Row(1, "amy", 10000.00, 1000.0, true))) + + val df7 = spark.read + .option("pushDownOffset", "false") + .table("h2.test.employee") + .where($"dept" === 1) + .sort($"salary") + .limit(2) + .offset(1) + checkLimitRemoved(df7) + checkOffsetRemoved(df7, false) + checkPushedInfo(df7, "PushedFilters: [DEPT IS NOT NULL, DEPT = 1]," + + " PushedTopN: ORDER BY [SALARY ASC NULLS FIRST] LIMIT 2, ReadSchema:") + checkAnswer(df7, Seq(Row(1, "amy", 10000.00, 1000.0, true))) + + val df8 = spark.read + .option("pushDownLimit", "false") + .option("pushDownOffset", "false") + .table("h2.test.employee") + .where($"dept" === 1) + .sort($"salary") + .limit(2) + .offset(1) + checkLimitRemoved(df8, false) + checkOffsetRemoved(df8, false) + checkPushedInfo(df8, "PushedFilters: [DEPT IS NOT NULL, DEPT = 1], ReadSchema:") + checkAnswer(df8, Seq(Row(1, "amy", 10000.00, 1000.0, true))) + + val df9 = spark.read + .option("partitionColumn", "dept") + .option("lowerBound", "0") + .option("upperBound", "2") + .option("numPartitions", "2") + .table("h2.test.employee") + .filter($"dept" > 1) + .limit(2) + .offset(1) + checkLimitRemoved(df9, false) + checkOffsetRemoved(df9, false) + checkPushedInfo(df9, + "PushedFilters: [DEPT IS NOT NULL, DEPT > 1], PushedLimit: LIMIT 2, ReadSchema:") + checkAnswer(df9, Seq(Row(2, "david", 10000.00, 1300.0, true))) + + val df10 = spark.read + .table("h2.test.employee") + .groupBy("DEPT").sum("SALARY") + .limit(2) + .offset(1) + checkLimitRemoved(df10, false) + checkOffsetRemoved(df10, false) + checkPushedInfo(df10, + "PushedAggregates: [SUM(SALARY)], PushedFilters: [], PushedGroupByExpressions: [DEPT], ") + checkAnswer(df10, Seq(Row(2, 22000.00))) + + val name = udf { (x: String) => x.matches("cat|dav|amy") } + val sub = udf { (x: String) => x.substring(0, 3) } + val df11 = spark.read + .table("h2.test.employee") + .select($"SALARY", $"BONUS", sub($"NAME").as("shortName")) + .filter(name($"shortName")) + .limit(2) + .offset(1) + checkLimitRemoved(df11, false) + checkOffsetRemoved(df11, false) + checkPushedInfo(df11, "PushedFilters: [], ") + checkAnswer(df11, Seq(Row(9000.00, 1200.0, "cat"))) + } + + test("simple scan with OFFSET and LIMIT") { + val df1 = spark.read + .table("h2.test.employee") + .where($"dept" === 1) + .offset(1) + .limit(1) + checkLimitRemoved(df1) + checkOffsetRemoved(df1) + checkPushedInfo(df1, + "[DEPT IS NOT NULL, DEPT = 1], PushedLimit: LIMIT 2, PushedOffset: OFFSET 1,") + checkAnswer(df1, Seq(Row(1, "cathy", 9000.00, 1200.0, false))) + + val df2 = spark.read + .option("pushDownOffset", "false") + .table("h2.test.employee") + .where($"dept" === 1) + .offset(1) + .limit(1) + checkLimitRemoved(df2) + checkOffsetRemoved(df2, false) + checkPushedInfo(df2, + "[DEPT IS NOT NULL, DEPT = 1], PushedLimit: LIMIT 2, ReadSchema:") + checkAnswer(df2, Seq(Row(1, "cathy", 9000.00, 1200.0, false))) + + val df3 = spark.read + .option("pushDownLimit", "false") + .table("h2.test.employee") + .where($"dept" === 1) + .offset(1) + .limit(1) + checkLimitRemoved(df3, false) + checkOffsetRemoved(df3) + checkPushedInfo(df3, + "[DEPT IS NOT NULL, DEPT = 1], PushedOffset: OFFSET 1, ReadSchema:") + checkAnswer(df3, Seq(Row(1, "cathy", 9000.00, 1200.0, false))) + + val df4 = spark.read + .option("pushDownOffset", "false") + .option("pushDownLimit", "false") + .table("h2.test.employee") + .where($"dept" === 1) + .offset(1) + .limit(1) + checkLimitRemoved(df4, false) + checkOffsetRemoved(df4, false) + checkPushedInfo(df4, + "[DEPT IS NOT NULL, DEPT = 1], ReadSchema:") + checkAnswer(df4, Seq(Row(1, "cathy", 9000.00, 1200.0, false))) + + val df5 = spark.read + .table("h2.test.employee") + .where($"dept" === 1) + .sort($"salary") + .offset(1) + .limit(1) + checkLimitRemoved(df5) + checkOffsetRemoved(df5) + checkPushedInfo(df5, "PushedFilters: [DEPT IS NOT NULL, DEPT = 1], " + + "PushedOffset: OFFSET 1, PushedTopN: ORDER BY [SALARY ASC NULLS FIRST] LIMIT 2, ReadSchema:") + checkAnswer(df5, Seq(Row(1, "amy", 10000.00, 1000.0, true))) + + val df6 = spark.read + .option("pushDownOffset", "false") + .table("h2.test.employee") + .where($"dept" === 1) + .sort($"salary") + .offset(1) + .limit(1) + checkLimitRemoved(df6) + checkOffsetRemoved(df6, false) + checkPushedInfo(df6, "[DEPT IS NOT NULL, DEPT = 1]," + + " PushedTopN: ORDER BY [SALARY ASC NULLS FIRST] LIMIT 2, ReadSchema:") + checkAnswer(df6, Seq(Row(1, "amy", 10000.00, 1000.0, true))) + + val df7 = spark.read + .option("pushDownLimit", "false") + .table("h2.test.employee") + .where($"dept" === 1) + .sort($"salary") + .offset(1) + .limit(1) + checkLimitRemoved(df7, false) + checkOffsetRemoved(df7, false) + checkPushedInfo(df7, "PushedFilters: [DEPT IS NOT NULL, DEPT = 1], ReadSchema:") + checkAnswer(df7, Seq(Row(1, "amy", 10000.00, 1000.0, true))) + + val df8 = spark.read + .option("pushDownOffset", "false") + .option("pushDownLimit", "false") + .table("h2.test.employee") + .where($"dept" === 1) + .sort($"salary") + .offset(1) + .limit(1) + checkLimitRemoved(df8, false) + checkOffsetRemoved(df8, false) + checkPushedInfo(df8, "PushedFilters: [DEPT IS NOT NULL, DEPT = 1], ReadSchema:") + checkAnswer(df8, Seq(Row(1, "amy", 10000.00, 1000.0, true))) + + val df9 = spark.read + .option("partitionColumn", "dept") + .option("lowerBound", "0") + .option("upperBound", "2") + .option("numPartitions", "2") + .table("h2.test.employee") + .filter($"dept" > 1) + .offset(1) + .limit(1) + checkLimitRemoved(df9, false) + checkOffsetRemoved(df9, false) + checkPushedInfo(df9, + "PushedFilters: [DEPT IS NOT NULL, DEPT > 1], PushedLimit: LIMIT 2, ReadSchema:") + checkAnswer(df9, Seq(Row(2, "david", 10000.00, 1300.0, true))) + + val df10 = sql("SELECT dept, sum(salary) FROM h2.test.employee group by dept LIMIT 1 OFFSET 1") + checkLimitRemoved(df10, false) + checkOffsetRemoved(df10, false) + checkPushedInfo(df10, + "PushedAggregates: [SUM(SALARY)], PushedFilters: [], PushedGroupByExpressions: [DEPT], ") + checkAnswer(df10, Seq(Row(2, 22000.00))) + + val name = udf { (x: String) => x.matches("cat|dav|amy") } + val sub = udf { (x: String) => x.substring(0, 3) } + val df11 = spark.read + .table("h2.test.employee") + .select($"SALARY", $"BONUS", sub($"NAME").as("shortName")) + .filter(name($"shortName")) + .offset(1) + .limit(1) + checkLimitRemoved(df11, false) + checkOffsetRemoved(df11, false) + checkPushedInfo(df11, "PushedFilters: [], ") + checkAnswer(df11, Seq(Row(9000.00, 1200.0, "cat"))) + } + private def checkSortRemoved(df: DataFrame, removed: Boolean = true): Unit = { val sorts = df.queryExecution.optimizedPlan.collect { case s: Sort => s @@ -225,7 +576,7 @@ class JDBCV2Suite extends QueryTest with SharedSparkSession with ExplainSuiteHel checkSortRemoved(df1) checkLimitRemoved(df1) checkPushedInfo(df1, - "PushedFilters: [], PushedTopN: ORDER BY [salary ASC NULLS FIRST] LIMIT 1, ") + "PushedFilters: [], PushedTopN: ORDER BY [SALARY ASC NULLS FIRST] LIMIT 1, ") checkAnswer(df1, Seq(Row(1, "cathy", 9000.00, 1200.0, false))) val df2 = spark.read @@ -240,7 +591,7 @@ class JDBCV2Suite extends QueryTest with SharedSparkSession with ExplainSuiteHel checkSortRemoved(df2) checkLimitRemoved(df2) checkPushedInfo(df2, "PushedFilters: [DEPT IS NOT NULL, DEPT = 1], " + - "PushedTopN: ORDER BY [salary ASC NULLS FIRST] LIMIT 1, ") + "PushedTopN: ORDER BY [SALARY ASC NULLS FIRST] LIMIT 1, ") checkAnswer(df2, Seq(Row(1, "cathy", 9000.00, 1200.0, false))) val df3 = spark.read @@ -255,7 +606,7 @@ class JDBCV2Suite extends QueryTest with SharedSparkSession with ExplainSuiteHel checkSortRemoved(df3, false) checkLimitRemoved(df3, false) checkPushedInfo(df3, "PushedFilters: [DEPT IS NOT NULL, DEPT > 1], " + - "PushedTopN: ORDER BY [salary DESC NULLS LAST] LIMIT 1, ") + "PushedTopN: ORDER BY [SALARY DESC NULLS LAST] LIMIT 1, ") checkAnswer(df3, Seq(Row(2, "alex", 12000.00, 1200.0, false))) val df4 = @@ -264,7 +615,7 @@ class JDBCV2Suite extends QueryTest with SharedSparkSession with ExplainSuiteHel checkSortRemoved(df4) checkLimitRemoved(df4) checkPushedInfo(df4, "PushedFilters: [DEPT IS NOT NULL, DEPT > 1], " + - "PushedTopN: ORDER BY [salary ASC NULLS LAST] LIMIT 1, ") + "PushedTopN: ORDER BY [SALARY ASC NULLS LAST] LIMIT 1, ") checkAnswer(df4, Seq(Row("david"))) val df5 = spark.read.table("h2.test.employee") @@ -307,6 +658,38 @@ class JDBCV2Suite extends QueryTest with SharedSparkSession with ExplainSuiteHel checkLimitRemoved(df8, false) checkPushedInfo(df8, "PushedFilters: [], ") checkAnswer(df8, Seq(Row(2, "alex", 12000.00, 1200.0, false))) + + val df9 = spark.read + .table("h2.test.employee") + .select($"DEPT", $"name", $"SALARY", + when(($"SALARY" > 8000).and($"SALARY" < 10000), $"salary").otherwise(0).as("key")) + .sort("key", "dept", "SALARY") + .limit(3) + checkSortRemoved(df9) + checkLimitRemoved(df9) + checkPushedInfo(df9, "PushedFilters: [], " + + "PushedTopN: ORDER BY [CASE WHEN (SALARY > 8000.00) AND " + + "(SALARY < 10000.00) THEN SALARY ELSE 0.00 END ASC NULL..., ") + checkAnswer(df9, + Seq(Row(1, "amy", 10000, 0), Row(2, "david", 10000, 0), Row(2, "alex", 12000, 0))) + + val df10 = spark.read + .option("partitionColumn", "dept") + .option("lowerBound", "0") + .option("upperBound", "2") + .option("numPartitions", "2") + .table("h2.test.employee") + .select($"DEPT", $"name", $"SALARY", + when(($"SALARY" > 8000).and($"SALARY" < 10000), $"salary").otherwise(0).as("key")) + .orderBy($"key", $"dept", $"SALARY") + .limit(3) + checkSortRemoved(df10, false) + checkLimitRemoved(df10, false) + checkPushedInfo(df10, "PushedFilters: [], " + + "PushedTopN: ORDER BY [CASE WHEN (SALARY > 8000.00) AND " + + "(SALARY < 10000.00) THEN SALARY ELSE 0.00 END ASC NULL..., ") + checkAnswer(df10, + Seq(Row(1, "amy", 10000, 0), Row(2, "david", 10000, 0), Row(2, "alex", 12000, 0))) } test("simple scan with top N: order by with alias") { @@ -393,6 +776,94 @@ class JDBCV2Suite extends QueryTest with SharedSparkSession with ExplainSuiteHel checkFiltersRemoved(df10) checkPushedInfo(df10, "PushedFilters: [ID IS NOT NULL, ID > 1], ") checkAnswer(df10, Row("mary", 2)) + + val df11 = sql( + """ + |SELECT * FROM h2.test.employee + |WHERE GREATEST(bonus, 1100) > 1200 AND LEAST(salary, 10000) > 9000 AND RAND(1) < 1 + |""".stripMargin) + checkFiltersRemoved(df11) + checkPushedInfo(df11, "PushedFilters: " + + "[(GREATEST(BONUS, 1100.0)) > 1200.0, (LEAST(SALARY, 10000.00)) > 9000.00, RAND(1) < 1.0]") + checkAnswer(df11, Row(2, "david", 10000, 1300, true)) + + val df12 = sql( + """ + |SELECT * FROM h2.test.employee + |WHERE IF(SALARY > 10000, SALARY, LEAST(SALARY, 1000)) > 1200 + |""".stripMargin) + checkFiltersRemoved(df12) + checkPushedInfo(df12, "PushedFilters: " + + "[(CASE WHEN SALARY > 10000.00 THEN SALARY ELSE LEAST(SALARY, 1000.00) END) > 1200.00]") + checkAnswer(df12, Seq(Row(2, "alex", 12000, 1200, false), Row(6, "jen", 12000, 1200, true))) + + val df13 = spark.table("h2.test.employee") + .filter(logarithm($"bonus") > 7) + .filter(exp($"bonus") > 0) + .filter(pow($"bonus", 2) === 1440000) + .filter(sqrt($"bonus") > 34) + .filter(floor($"bonus") === 1200) + .filter(ceil($"bonus") === 1200) + checkFiltersRemoved(df13) + checkPushedInfo(df13, "PushedFilters: [BONUS IS NOT NULL, LN(BONUS) > 7.0, EXP(BONUS) > 0.0, " + + "(POWER(BONUS, 2.0)) = 1440000.0, SQRT(BONU...,") + checkAnswer(df13, Seq(Row(1, "cathy", 9000, 1200, false), + Row(2, "alex", 12000, 1200, false), Row(6, "jen", 12000, 1200, true))) + + // H2 does not support width_bucket + val df14 = sql( + """ + |SELECT * FROM h2.test.employee + |WHERE width_bucket(bonus, 1, 6, 3) > 4 + |""".stripMargin) + checkFiltersRemoved(df14, false) + checkPushedInfo(df14, "PushedFilters: [BONUS IS NOT NULL]") + checkAnswer(df14, Seq.empty[Row]) + + val df15 = spark.table("h2.test.employee") + .filter(logarithm(2, $"bonus") > 10) + .filter(log10($"bonus") > 3) + .filter(round($"bonus") === 1200) + .filter(degrees($"bonus") > 68754) + .filter(radians($"bonus") > 20) + .filter(signum($"bonus") === 1) + checkFiltersRemoved(df15) + checkPushedInfo(df15, "PushedFilters: [BONUS IS NOT NULL, (LOG(2.0, BONUS)) > 10.0, " + + "LOG10(BONUS) > 3.0, (ROUND(BONUS, 0)) = 1200.0, DEG...,") + checkAnswer(df15, Seq(Row(1, "cathy", 9000, 1200, false), + Row(2, "alex", 12000, 1200, false), Row(6, "jen", 12000, 1200, true))) + + val df16 = spark.table("h2.test.employee") + .filter(sin($"bonus") < -0.08) + .filter(sinh($"bonus") > 200) + .filter(cos($"bonus") > 0.9) + .filter(cosh($"bonus") > 200) + .filter(tan($"bonus") < -0.08) + .filter(tanh($"bonus") === 1) + .filter(asin($"bonus") > 0.1) + .filter(acos($"bonus") > 1.4) + .filter(atan($"bonus") > 1.4) + .filter(atan2($"bonus", $"bonus") > 0.7) + checkFiltersRemoved(df16) + checkPushedInfo(df16, "PushedFilters: [BONUS IS NOT NULL, SIN(BONUS) < -0.08, " + + "SINH(BONUS) > 200.0, COS(BONUS) > 0.9, COSH(BONUS) > 200....,") + checkAnswer(df16, Seq(Row(1, "cathy", 9000, 1200, false), + Row(2, "alex", 12000, 1200, false), Row(6, "jen", 12000, 1200, true))) + + // H2 does not support log2, asinh, acosh, atanh, cbrt + val df17 = sql( + """ + |SELECT * FROM h2.test.employee + |WHERE log2(dept) > 2.5 + |AND asinh(bonus / salary) > 0.09 + |AND acosh(dept) > 2.4 + |AND atanh(bonus / salary) > 0.1 + |AND cbrt(dept) > 1.8 + |""".stripMargin) + checkFiltersRemoved(df17, false) + checkPushedInfo(df17, + "PushedFilters: [DEPT IS NOT NULL, BONUS IS NOT NULL, SALARY IS NOT NULL]") + checkAnswer(df17, Seq(Row(6, "jen", 12000, 1200, true))) } test("scan with filter push-down with ansi mode") { @@ -426,10 +897,11 @@ class JDBCV2Suite extends QueryTest with SharedSparkSession with ExplainSuiteHel checkAnswer(df2, Seq.empty) } - val df3 = sql(""" - |SELECT * FROM h2.test.employee - |WHERE (CASE WHEN SALARY > 10000 THEN BONUS ELSE BONUS + 200 END) > 1200 - |""".stripMargin) + val df3 = sql( + """ + |SELECT * FROM h2.test.employee + |WHERE (CASE WHEN SALARY > 10000 THEN BONUS ELSE BONUS + 200 END) > 1200 + |""".stripMargin) checkFiltersRemoved(df3, ansiMode) val expectedPlanFragment3 = if (ansiMode) { @@ -469,46 +941,22 @@ class JDBCV2Suite extends QueryTest with SharedSparkSession with ExplainSuiteHel checkAnswer(df5, Seq(Row(1, "amy", 10000, 1000, true), Row(1, "cathy", 9000, 1200, false), Row(6, "jen", 12000, 1200, true))) - val df6 = spark.table("h2.test.employee") - .filter(ln($"dept") > 1) - .filter(exp($"salary") > 2000) - .filter(pow($"dept", 2) > 4) - .filter(sqrt($"salary") > 100) - checkFiltersRemoved(df6, ansiMode) - val expectedPlanFragment6 = if (ansiMode) { - "PushedFilters: [DEPT IS NOT NULL, SALARY IS NOT NULL, " + - "LN(CAST(DEPT AS double)) > 1.0, EXP(CAST(SALARY AS double)...," - } else { - "PushedFilters: [DEPT IS NOT NULL, SALARY IS NOT NULL]" - } - checkPushedInfo(df6, expectedPlanFragment6) - checkAnswer(df6, Seq(Row(6, "jen", 12000, 1200, true))) - - // H2 does not support width_bucket - val df7 = sql(""" - |SELECT * FROM h2.test.employee - |WHERE width_bucket(dept, 1, 6, 3) > 1 - |""".stripMargin) - checkFiltersRemoved(df7, false) - checkPushedInfo(df7, "PushedFilters: [DEPT IS NOT NULL]") - checkAnswer(df7, Seq(Row(6, "jen", 12000, 1200, true))) - - val df8 = sql( + val df6 = sql( """ |SELECT * FROM h2.test.employee |WHERE cast(bonus as string) like '%30%' |AND cast(dept as byte) > 1 |AND cast(dept as short) > 1 |AND cast(bonus as decimal(20, 2)) > 1200""".stripMargin) - checkFiltersRemoved(df8, ansiMode) + checkFiltersRemoved(df6, ansiMode) val expectedPlanFragment8 = if (ansiMode) { "PushedFilters: [BONUS IS NOT NULL, DEPT IS NOT NULL, " + "CAST(BONUS AS string) LIKE '%30%', CAST(DEPT AS byte) > 1, ...," } else { "PushedFilters: [BONUS IS NOT NULL, DEPT IS NOT NULL]," } - checkPushedInfo(df8, expectedPlanFragment8) - checkAnswer(df8, Seq(Row(2, "david", 10000, 1300, true))) + checkPushedInfo(df6, expectedPlanFragment8) + checkAnswer(df6, Seq(Row(2, "david", 10000, 1300, true))) } } } @@ -1251,11 +1699,21 @@ class JDBCV2Suite extends QueryTest with SharedSparkSession with ExplainSuiteHel } test("column name with composite field") { - checkAnswer(sql("SELECT dept_id FROM h2.test.dept"), Seq(Row(1), Row(2))) - val df = sql("SELECT COUNT(dept_id) FROM h2.test.dept") - checkAggregateRemoved(df) - checkPushedInfo(df, "PushedAggregates: [COUNT(dept_id)]") - checkAnswer(df, Seq(Row(2))) + checkAnswer(sql("SELECT `dept_id`, `dept_id1` FROM h2.test.dept"), Seq(Row(1, 1), Row(2, 1))) + + val df1 = sql("SELECT COUNT(`dept_id`) FROM h2.test.dept") + checkPushedInfo(df1, "PushedAggregates: [COUNT(dept_id)]") + checkAnswer(df1, Seq(Row(2))) + + val df2 = sql("SELECT `dept_id1`, COUNT(`dept_id`) FROM h2.test.dept GROUP BY `dept_id1`") + checkPushedInfo(df2, + "PushedGroupByExpressions: [dept_id1]", "PushedAggregates: [COUNT(dept_id)]") + checkAnswer(df2, Seq(Row(1, 2))) + + val df3 = sql("SELECT `dept_id`, COUNT(`dept_id1`) FROM h2.test.dept GROUP BY `dept_id`") + checkPushedInfo(df3, + "PushedGroupByExpressions: [dept_id]", "PushedAggregates: [COUNT(dept_id1)]") + checkAnswer(df3, Seq(Row(1, 1), Row(2, 1))) } test("column name with non-ascii") {