From a071826f72cd717a58bf37b877f805490f7a147f Mon Sep 17 00:00:00 2001 From: fwang12 Date: Tue, 5 Jan 2021 15:55:30 +0900 Subject: [PATCH 01/29] [SPARK-33100][SQL] Ignore a semicolon inside a bracketed comment in spark-sql ### What changes were proposed in this pull request? Now the spark-sql does not support parse the sql statements with bracketed comments. For the sql statements: ``` /* SELECT 'test'; */ SELECT 'test'; ``` Would be split to two statements: The first one: `/* SELECT 'test'` The second one: `*/ SELECT 'test'` Then it would throw an exception because the first one is illegal. In this PR, we ignore the content in bracketed comments while splitting the sql statements. Besides, we ignore the comment without any content. ### Why are the changes needed? Spark-sql might split the statements inside bracketed comments and it is not correct. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Added UT. Closes #29982 from turboFei/SPARK-33110. Lead-authored-by: fwang12 Co-authored-by: turbofei Signed-off-by: Takeshi Yamamuro --- .../hive/thriftserver/SparkSQLCLIDriver.scala | 40 +++++++++++++++---- .../sql/hive/thriftserver/CliSuite.scala | 23 +++++++++++ 2 files changed, 55 insertions(+), 8 deletions(-) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala index f2fd373bf6cc0..9155eacfa4896 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala @@ -522,14 +522,22 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging { // Note: [SPARK-31595] if there is a `'` in a double quoted string, or a `"` in a single quoted // string, the origin implementation from Hive will not drop the trailing semicolon as expected, // hence we refined this function a little bit. + // Note: [SPARK-33100] Ignore a semicolon inside a bracketed comment in spark-sql. private def splitSemiColon(line: String): JList[String] = { var insideSingleQuote = false var insideDoubleQuote = false - var insideComment = false + var insideSimpleComment = false + var bracketedCommentLevel = 0 var escape = false var beginIndex = 0 + var includingStatement = false val ret = new JArrayList[String] + def insideBracketedComment: Boolean = bracketedCommentLevel > 0 + def insideComment: Boolean = insideSimpleComment || insideBracketedComment + def statementBegin(index: Int): Boolean = includingStatement || (!insideComment && + index > beginIndex && !s"${line.charAt(index)}".trim.isEmpty) + for (index <- 0 until line.length) { if (line.charAt(index) == '\'' && !insideComment) { // take a look to see if it is escaped @@ -553,21 +561,33 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging { // Sample query: select "quoted value --" // ^^ avoids starting a comment if it's inside quotes. } else if (hasNext && line.charAt(index + 1) == '-') { - // ignore quotes and ; - insideComment = true + // ignore quotes and ; in simple comment + insideSimpleComment = true } } else if (line.charAt(index) == ';') { if (insideSingleQuote || insideDoubleQuote || insideComment) { // do not split } else { - // split, do not include ; itself - ret.add(line.substring(beginIndex, index)) + if (includingStatement) { + // split, do not include ; itself + ret.add(line.substring(beginIndex, index)) + } beginIndex = index + 1 + includingStatement = false } } else if (line.charAt(index) == '\n') { - // with a new line the inline comment should end. + // with a new line the inline simple comment should end. if (!escape) { - insideComment = false + insideSimpleComment = false + } + } else if (line.charAt(index) == '/' && !insideSimpleComment) { + val hasNext = index + 1 < line.length + if (insideSingleQuote || insideDoubleQuote) { + // Ignores '/' in any case of quotes + } else if (insideBracketedComment && line.charAt(index - 1) == '*' ) { + bracketedCommentLevel -= 1 + } else if (hasNext && !insideBracketedComment && line.charAt(index + 1) == '*') { + bracketedCommentLevel += 1 } } // set the escape @@ -576,8 +596,12 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging { } else if (line.charAt(index) == '\\') { escape = true } + + includingStatement = statementBegin(index) + } + if (includingStatement) { + ret.add(line.substring(beginIndex)) } - ret.add(line.substring(beginIndex)) ret } } diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala index d39b94503fe40..6708cf99e7f41 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala @@ -571,4 +571,27 @@ class CliSuite extends SparkFunSuite with BeforeAndAfterAll with Logging { // the date formatter for `java.sql.LocalDate` must output negative years with sign. runCliWithin(1.minute)("SELECT MAKE_DATE(-44, 3, 15);" -> "-0044-03-15") } + + test("SPARK-33100: Ignore a semicolon inside a bracketed comment in spark-sql") { + runCliWithin(4.minute)( + "/* SELECT 'test';*/ SELECT 'test';" -> "test", + ";;/* SELECT 'test';*/ SELECT 'test';" -> "test", + "/* SELECT 'test';*/;; SELECT 'test';" -> "test", + "SELECT 'test'; -- SELECT 'test';" -> "", + "SELECT 'test'; /* SELECT 'test';*/;" -> "", + "/*$meta chars{^\\;}*/ SELECT 'test';" -> "test", + "/*\nmulti-line\n*/ SELECT 'test';" -> "test", + "/*/* multi-level bracketed*/ SELECT 'test';" -> "test" + ) + } + + test("SPARK-33100: test sql statements with hint in bracketed comment") { + runCliWithin(2.minute)( + "CREATE TEMPORARY VIEW t1 AS SELECT * FROM VALUES(1, 2) AS t1(k, v);" -> "", + "CREATE TEMPORARY VIEW t2 AS SELECT * FROM VALUES(2, 1) AS t2(k, v);" -> "", + "EXPLAIN SELECT /*+ MERGEJOIN(t1) */ t1.* FROM t1 JOIN t2 ON t1.k = t2.v;" -> "SortMergeJoin", + "EXPLAIN SELECT /* + MERGEJOIN(t1) */ t1.* FROM t1 JOIN t2 ON t1.k = t2.v;" + -> "BroadcastHashJoin" + ) + } } From f252a9334e49dc359dd9255fcfe17a6bc75b8781 Mon Sep 17 00:00:00 2001 From: "tanel.kiis@gmail.com" Date: Tue, 5 Jan 2021 16:00:24 +0900 Subject: [PATCH 02/29] [SPARK-33935][SQL] Fix CBO cost function ### What changes were proposed in this pull request? Changed the cost function in CBO to match documentation. ### Why are the changes needed? The parameter `spark.sql.cbo.joinReorder.card.weight` is documented as: ``` The weight of cardinality (number of rows) for plan cost comparison in join reorder: rows * weight + size * (1 - weight). ``` The implementation in `JoinReorderDP.betterThan` does not match this documentaiton: ``` def betterThan(other: JoinPlan, conf: SQLConf): Boolean = { if (other.planCost.card == 0 || other.planCost.size == 0) { false } else { val relativeRows = BigDecimal(this.planCost.card) / BigDecimal(other.planCost.card) val relativeSize = BigDecimal(this.planCost.size) / BigDecimal(other.planCost.size) relativeRows * conf.joinReorderCardWeight + relativeSize * (1 - conf.joinReorderCardWeight) < 1 } } ``` This different implementation has an unfortunate consequence: given two plans A and B, both A betterThan B and B betterThan A might give the same results. This happes when one has many rows with small sizes and other has few rows with large sizes. A example values, that have this fenomen with the default weight value (0.7): A.card = 500, B.card = 300 A.size = 30, B.size = 80 Both A betterThan B and B betterThan A would have score above 1 and would return false. This happens with several of the TPCDS queries. The new implementation does not have this behavior. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? New and existing UTs Closes #30965 from tanelk/SPARK-33935_cbo_cost_function. Authored-by: tanel.kiis@gmail.com Signed-off-by: Takeshi Yamamuro --- .../optimizer/CostBasedJoinReorder.scala | 13 +- .../joinReorder/JoinReorderSuite.scala | 15 + .../StarJoinCostBasedReorderSuite.scala | 8 +- .../approved-plans-v1_4/q13.sf100/explain.txt | 132 ++-- .../q13.sf100/simplified.txt | 34 +- .../approved-plans-v1_4/q17.sf100/explain.txt | 194 +++--- .../q17.sf100/simplified.txt | 130 ++-- .../approved-plans-v1_4/q18.sf100/explain.txt | 158 ++--- .../q18.sf100/simplified.txt | 50 +- .../approved-plans-v1_4/q19.sf100/explain.txt | 368 +++++------ .../q19.sf100/simplified.txt | 116 ++-- .../q24a.sf100/explain.txt | 118 ++-- .../q24a.sf100/simplified.txt | 34 +- .../q24b.sf100/explain.txt | 118 ++-- .../q24b.sf100/simplified.txt | 34 +- .../approved-plans-v1_4/q25.sf100/explain.txt | 194 +++--- .../q25.sf100/simplified.txt | 130 ++-- .../approved-plans-v1_4/q33.sf100/explain.txt | 264 ++++---- .../q33.sf100/simplified.txt | 58 +- .../approved-plans-v1_4/q52.sf100/explain.txt | 138 ++--- .../q52.sf100/simplified.txt | 26 +- .../approved-plans-v1_4/q55.sf100/explain.txt | 134 ++-- .../q55.sf100/simplified.txt | 26 +- .../approved-plans-v1_4/q72.sf100/explain.txt | 264 ++++---- .../q72.sf100/simplified.txt | 150 ++--- .../approved-plans-v1_4/q81.sf100/explain.txt | 570 +++++++++--------- .../q81.sf100/simplified.txt | 142 ++--- .../approved-plans-v1_4/q91.sf100/explain.txt | 306 +++++----- .../q91.sf100/simplified.txt | 62 +- .../q18a.sf100/explain.txt | 306 +++++----- .../q18a.sf100/simplified.txt | 54 +- .../approved-plans-v2_7/q72.sf100/explain.txt | 264 ++++---- .../q72.sf100/simplified.txt | 150 ++--- 33 files changed, 2386 insertions(+), 2374 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala index 11b675e75869e..c41686da79487 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala @@ -349,14 +349,11 @@ object JoinReorderDP extends PredicateHelper with Logging { } def betterThan(other: JoinPlan, conf: SQLConf): Boolean = { - if (other.planCost.card == 0 || other.planCost.size == 0) { - false - } else { - val relativeRows = BigDecimal(this.planCost.card) / BigDecimal(other.planCost.card) - val relativeSize = BigDecimal(this.planCost.size) / BigDecimal(other.planCost.size) - relativeRows * conf.joinReorderCardWeight + - relativeSize * (1 - conf.joinReorderCardWeight) < 1 - } + val thisCost = BigDecimal(this.planCost.card) * conf.joinReorderCardWeight + + BigDecimal(this.planCost.size) * (1 - conf.joinReorderCardWeight) + val otherCost = BigDecimal(other.planCost.card) * conf.joinReorderCardWeight + + BigDecimal(other.planCost.size) * (1 - conf.joinReorderCardWeight) + thisCost < otherCost } } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/joinReorder/JoinReorderSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/joinReorder/JoinReorderSuite.scala index b84207397e5cc..2e1cf4a137e25 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/joinReorder/JoinReorderSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/joinReorder/JoinReorderSuite.scala @@ -21,6 +21,7 @@ import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap} import org.apache.spark.sql.catalyst.optimizer._ +import org.apache.spark.sql.catalyst.optimizer.JoinReorderDP.JoinPlan import org.apache.spark.sql.catalyst.plans.{Cross, Inner} import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, LocalRelation, LogicalPlan} import org.apache.spark.sql.catalyst.rules.RuleExecutor @@ -363,4 +364,18 @@ class JoinReorderSuite extends JoinReorderPlanTestBase with StatsEstimationTestB assertEqualJoinPlans(Optimize, originalPlan3, bestPlan3) } + + test("SPARK-33935: betterThan should be consistent") { + val plan1 = JoinPlan(null, null, null, Cost(300, 80)) + val plan2 = JoinPlan(null, null, null, Cost(500, 30)) + + // cost1 = 300*0.7 + 80*0.3 = 234 + // cost2 = 500*0.7 + 30*0.3 = 359 + + assert(!plan1.betterThan(plan1, conf)) + assert(!plan2.betterThan(plan2, conf)) + + assert(plan1.betterThan(plan2, conf)) + assert(!plan2.betterThan(plan1, conf)) + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/joinReorder/StarJoinCostBasedReorderSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/joinReorder/StarJoinCostBasedReorderSuite.scala index 703be48c6a2a9..a42914765dcc8 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/joinReorder/StarJoinCostBasedReorderSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/joinReorder/StarJoinCostBasedReorderSuite.scala @@ -294,12 +294,12 @@ class StarJoinCostBasedReorderSuite extends JoinReorderPlanTestBase with StatsEs (nameToAttr("f1_fk2") === nameToAttr("d2_pk"))) val expected = - f1.join(d2, Inner, Some(nameToAttr("f1_fk2") === nameToAttr("d2_pk"))) - .join(d1, Inner, Some(nameToAttr("f1_fk1") === nameToAttr("d1_pk"))) - .join(t3.join(t4, Inner, Some(nameToAttr("t3_c1") === nameToAttr("t4_c1"))), Inner, - Some(nameToAttr("t3_c1") === nameToAttr("t4_c1"))) + t3.join(t4, Inner, Some(nameToAttr("t3_c1") === nameToAttr("t4_c1"))) .join(t1.join(t2, Inner, Some(nameToAttr("t1_c1") === nameToAttr("t2_c1"))), Inner, Some(nameToAttr("t1_c2") === nameToAttr("t4_c2"))) + .join(f1 + .join(d2, Inner, Some(nameToAttr("f1_fk2") === nameToAttr("d2_pk"))) + .join(d1, Inner, Some(nameToAttr("f1_fk1") === nameToAttr("d1_pk")))) .select(outputsOf(d1, t1, t2, t3, t4, f1, d2): _*) assertEqualJoinPlans(Optimize, query, expected) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.sf100/explain.txt index 8ee427262b332..327e7db702faa 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.sf100/explain.txt @@ -4,8 +4,8 @@ +- * HashAggregate (36) +- * Project (35) +- * BroadcastHashJoin Inner BuildRight (34) - :- * Project (28) - : +- * BroadcastHashJoin Inner BuildRight (27) + :- * Project (29) + : +- * BroadcastHashJoin Inner BuildRight (28) : :- * Project (22) : : +- * BroadcastHashJoin Inner BuildRight (21) : : :- * Project (15) @@ -27,16 +27,16 @@ : : +- * Project (19) : : +- * Filter (18) : : +- * ColumnarToRow (17) - : : +- Scan parquet default.date_dim (16) - : +- BroadcastExchange (26) - : +- * Filter (25) - : +- * ColumnarToRow (24) - : +- Scan parquet default.store (23) + : : +- Scan parquet default.customer_address (16) + : +- BroadcastExchange (27) + : +- * Project (26) + : +- * Filter (25) + : +- * ColumnarToRow (24) + : +- Scan parquet default.date_dim (23) +- BroadcastExchange (33) - +- * Project (32) - +- * Filter (31) - +- * ColumnarToRow (30) - +- Scan parquet default.customer_address (29) + +- * Filter (32) + +- * ColumnarToRow (31) + +- Scan parquet default.store (30) (1) Scan parquet default.store_sales @@ -107,94 +107,94 @@ Join condition: (((((((cd_marital_status#12 = M) AND (cd_education_status#13 = A Output [7]: [ss_sold_date_sk#1, ss_addr_sk#4, ss_store_sk#5, ss_quantity#6, ss_ext_sales_price#8, ss_ext_wholesale_cost#9, ss_net_profit#10] Input [13]: [ss_sold_date_sk#1, ss_hdemo_sk#3, ss_addr_sk#4, ss_store_sk#5, ss_quantity#6, ss_sales_price#7, ss_ext_sales_price#8, ss_ext_wholesale_cost#9, ss_net_profit#10, cd_marital_status#12, cd_education_status#13, hd_demo_sk#15, hd_dep_count#16] -(16) Scan parquet default.date_dim -Output [2]: [d_date_sk#18, d_year#19] +(16) Scan parquet default.customer_address +Output [3]: [ca_address_sk#18, ca_state#19, ca_country#20] Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/customer_address] +PushedFilters: [IsNotNull(ca_country), EqualTo(ca_country,United States), IsNotNull(ca_address_sk), Or(Or(In(ca_state, [TX,OH]),In(ca_state, [OR,NM,KY])),In(ca_state, [VA,TX,MS]))] +ReadSchema: struct (17) ColumnarToRow [codegen id : 3] -Input [2]: [d_date_sk#18, d_year#19] +Input [3]: [ca_address_sk#18, ca_state#19, ca_country#20] (18) Filter [codegen id : 3] -Input [2]: [d_date_sk#18, d_year#19] -Condition : ((isnotnull(d_year#19) AND (d_year#19 = 2001)) AND isnotnull(d_date_sk#18)) +Input [3]: [ca_address_sk#18, ca_state#19, ca_country#20] +Condition : (((isnotnull(ca_country#20) AND (ca_country#20 = United States)) AND isnotnull(ca_address_sk#18)) AND ((ca_state#19 IN (TX,OH) OR ca_state#19 IN (OR,NM,KY)) OR ca_state#19 IN (VA,TX,MS))) (19) Project [codegen id : 3] -Output [1]: [d_date_sk#18] -Input [2]: [d_date_sk#18, d_year#19] +Output [2]: [ca_address_sk#18, ca_state#19] +Input [3]: [ca_address_sk#18, ca_state#19, ca_country#20] (20) BroadcastExchange -Input [1]: [d_date_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#20] +Input [2]: [ca_address_sk#18, ca_state#19] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#21] (21) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#1] -Right keys [1]: [d_date_sk#18] -Join condition: None +Left keys [1]: [ss_addr_sk#4] +Right keys [1]: [ca_address_sk#18] +Join condition: ((((ca_state#19 IN (TX,OH) AND (ss_net_profit#10 >= 100.00)) AND (ss_net_profit#10 <= 200.00)) OR ((ca_state#19 IN (OR,NM,KY) AND (ss_net_profit#10 >= 150.00)) AND (ss_net_profit#10 <= 300.00))) OR ((ca_state#19 IN (VA,TX,MS) AND (ss_net_profit#10 >= 50.00)) AND (ss_net_profit#10 <= 250.00))) (22) Project [codegen id : 6] -Output [6]: [ss_addr_sk#4, ss_store_sk#5, ss_quantity#6, ss_ext_sales_price#8, ss_ext_wholesale_cost#9, ss_net_profit#10] -Input [8]: [ss_sold_date_sk#1, ss_addr_sk#4, ss_store_sk#5, ss_quantity#6, ss_ext_sales_price#8, ss_ext_wholesale_cost#9, ss_net_profit#10, d_date_sk#18] +Output [5]: [ss_sold_date_sk#1, ss_store_sk#5, ss_quantity#6, ss_ext_sales_price#8, ss_ext_wholesale_cost#9] +Input [9]: [ss_sold_date_sk#1, ss_addr_sk#4, ss_store_sk#5, ss_quantity#6, ss_ext_sales_price#8, ss_ext_wholesale_cost#9, ss_net_profit#10, ca_address_sk#18, ca_state#19] -(23) Scan parquet default.store -Output [1]: [s_store_sk#21] +(23) Scan parquet default.date_dim +Output [2]: [d_date_sk#22, d_year#23] Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] +ReadSchema: struct (24) ColumnarToRow [codegen id : 4] -Input [1]: [s_store_sk#21] +Input [2]: [d_date_sk#22, d_year#23] (25) Filter [codegen id : 4] -Input [1]: [s_store_sk#21] -Condition : isnotnull(s_store_sk#21) +Input [2]: [d_date_sk#22, d_year#23] +Condition : ((isnotnull(d_year#23) AND (d_year#23 = 2001)) AND isnotnull(d_date_sk#22)) -(26) BroadcastExchange -Input [1]: [s_store_sk#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#22] +(26) Project [codegen id : 4] +Output [1]: [d_date_sk#22] +Input [2]: [d_date_sk#22, d_year#23] -(27) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_store_sk#5] -Right keys [1]: [s_store_sk#21] +(27) BroadcastExchange +Input [1]: [d_date_sk#22] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#24] + +(28) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_sold_date_sk#1] +Right keys [1]: [d_date_sk#22] Join condition: None -(28) Project [codegen id : 6] -Output [5]: [ss_addr_sk#4, ss_quantity#6, ss_ext_sales_price#8, ss_ext_wholesale_cost#9, ss_net_profit#10] -Input [7]: [ss_addr_sk#4, ss_store_sk#5, ss_quantity#6, ss_ext_sales_price#8, ss_ext_wholesale_cost#9, ss_net_profit#10, s_store_sk#21] +(29) Project [codegen id : 6] +Output [4]: [ss_store_sk#5, ss_quantity#6, ss_ext_sales_price#8, ss_ext_wholesale_cost#9] +Input [6]: [ss_sold_date_sk#1, ss_store_sk#5, ss_quantity#6, ss_ext_sales_price#8, ss_ext_wholesale_cost#9, d_date_sk#22] -(29) Scan parquet default.customer_address -Output [3]: [ca_address_sk#23, ca_state#24, ca_country#25] +(30) Scan parquet default.store +Output [1]: [s_store_sk#25] Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_country), EqualTo(ca_country,United States), IsNotNull(ca_address_sk), Or(Or(In(ca_state, [TX,OH]),In(ca_state, [OR,NM,KY])),In(ca_state, [VA,TX,MS]))] -ReadSchema: struct - -(30) ColumnarToRow [codegen id : 5] -Input [3]: [ca_address_sk#23, ca_state#24, ca_country#25] +Location [not included in comparison]/{warehouse_dir}/store] +PushedFilters: [IsNotNull(s_store_sk)] +ReadSchema: struct -(31) Filter [codegen id : 5] -Input [3]: [ca_address_sk#23, ca_state#24, ca_country#25] -Condition : (((isnotnull(ca_country#25) AND (ca_country#25 = United States)) AND isnotnull(ca_address_sk#23)) AND ((ca_state#24 IN (TX,OH) OR ca_state#24 IN (OR,NM,KY)) OR ca_state#24 IN (VA,TX,MS))) +(31) ColumnarToRow [codegen id : 5] +Input [1]: [s_store_sk#25] -(32) Project [codegen id : 5] -Output [2]: [ca_address_sk#23, ca_state#24] -Input [3]: [ca_address_sk#23, ca_state#24, ca_country#25] +(32) Filter [codegen id : 5] +Input [1]: [s_store_sk#25] +Condition : isnotnull(s_store_sk#25) (33) BroadcastExchange -Input [2]: [ca_address_sk#23, ca_state#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#26] +Input [1]: [s_store_sk#25] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#26] (34) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_addr_sk#4] -Right keys [1]: [ca_address_sk#23] -Join condition: ((((ca_state#24 IN (TX,OH) AND (ss_net_profit#10 >= 100.00)) AND (ss_net_profit#10 <= 200.00)) OR ((ca_state#24 IN (OR,NM,KY) AND (ss_net_profit#10 >= 150.00)) AND (ss_net_profit#10 <= 300.00))) OR ((ca_state#24 IN (VA,TX,MS) AND (ss_net_profit#10 >= 50.00)) AND (ss_net_profit#10 <= 250.00))) +Left keys [1]: [ss_store_sk#5] +Right keys [1]: [s_store_sk#25] +Join condition: None (35) Project [codegen id : 6] Output [3]: [ss_quantity#6, ss_ext_sales_price#8, ss_ext_wholesale_cost#9] -Input [7]: [ss_addr_sk#4, ss_quantity#6, ss_ext_sales_price#8, ss_ext_wholesale_cost#9, ss_net_profit#10, ca_address_sk#23, ca_state#24] +Input [5]: [ss_store_sk#5, ss_quantity#6, ss_ext_sales_price#8, ss_ext_wholesale_cost#9, s_store_sk#25] (36) HashAggregate [codegen id : 6] Input [3]: [ss_quantity#6, ss_ext_sales_price#8, ss_ext_wholesale_cost#9] @@ -205,7 +205,7 @@ Results [7]: [sum#34, count#35, sum#36, count#37, sum#38, count#39, sum#40] (37) Exchange Input [7]: [sum#34, count#35, sum#36, count#37, sum#38, count#39, sum#40] -Arguments: SinglePartition, true, [id=#41] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#41] (38) HashAggregate [codegen id : 7] Input [7]: [sum#34, count#35, sum#36, count#37, sum#38, count#39, sum#40] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.sf100/simplified.txt index b457788dbd0b2..45d6c8f3b0bae 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.sf100/simplified.txt @@ -5,11 +5,11 @@ WholeStageCodegen (7) WholeStageCodegen (6) HashAggregate [ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] [sum,count,sum,count,sum,count,sum,sum,count,sum,count,sum,count,sum] Project [ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] - BroadcastHashJoin [ss_addr_sk,ca_address_sk,ca_state,ss_net_profit] - Project [ss_addr_sk,ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_addr_sk,ss_store_sk,ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_store_sk,ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,ss_store_sk,ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] + BroadcastHashJoin [ss_addr_sk,ca_address_sk,ca_state,ss_net_profit] Project [ss_sold_date_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit] BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price,hd_dep_count] Project [ss_sold_date_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,cd_marital_status,cd_education_status] @@ -35,23 +35,23 @@ WholeStageCodegen (7) InputAdapter BroadcastExchange #4 WholeStageCodegen (3) - Project [d_date_sk] - Filter [d_year,d_date_sk] + Project [ca_address_sk,ca_state] + Filter [ca_country,ca_address_sk,ca_state] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] + Scan parquet default.customer_address [ca_address_sk,ca_state,ca_country] InputAdapter BroadcastExchange #5 WholeStageCodegen (4) - Filter [s_store_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store [s_store_sk] + Project [d_date_sk] + Filter [d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #6 WholeStageCodegen (5) - Project [ca_address_sk,ca_state] - Filter [ca_country,ca_address_sk,ca_state] - ColumnarToRow - InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_state,ca_country] + Filter [s_store_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store [s_store_sk] 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 a17356ae04a03..a9ab8c3690a00 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 @@ -5,57 +5,57 @@ TakeOrderedAndProject (57) +- * HashAggregate (54) +- * Project (53) +- * SortMergeJoin Inner (52) - :- * Sort (43) - : +- Exchange (42) - : +- * Project (41) - : +- * SortMergeJoin Inner (40) - : :- * Sort (27) - : : +- Exchange (26) - : : +- * Project (25) - : : +- * SortMergeJoin Inner (24) - : : :- * Sort (18) - : : : +- Exchange (17) - : : : +- * Project (16) - : : : +- * BroadcastHashJoin Inner BuildRight (15) - : : : :- * Project (10) - : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet default.store_sales (1) - : : : : +- BroadcastExchange (8) - : : : : +- * Project (7) - : : : : +- * Filter (6) - : : : : +- * ColumnarToRow (5) - : : : : +- Scan parquet default.date_dim (4) - : : : +- BroadcastExchange (14) - : : : +- * Filter (13) - : : : +- * ColumnarToRow (12) - : : : +- Scan parquet default.store (11) - : : +- * Sort (23) - : : +- Exchange (22) - : : +- * Filter (21) - : : +- * ColumnarToRow (20) - : : +- Scan parquet default.item (19) - : +- * Sort (39) - : +- Exchange (38) - : +- * Project (37) - : +- * BroadcastHashJoin Inner BuildRight (36) - : :- * Filter (30) - : : +- * ColumnarToRow (29) - : : +- Scan parquet default.store_returns (28) - : +- BroadcastExchange (35) - : +- * Project (34) - : +- * Filter (33) - : +- * ColumnarToRow (32) - : +- Scan parquet default.date_dim (31) + :- * Sort (27) + : +- Exchange (26) + : +- * Project (25) + : +- * SortMergeJoin Inner (24) + : :- * Sort (18) + : : +- Exchange (17) + : : +- * Project (16) + : : +- * BroadcastHashJoin Inner BuildRight (15) + : : :- * Project (10) + : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : :- * Filter (3) + : : : : +- * ColumnarToRow (2) + : : : : +- Scan parquet default.store_sales (1) + : : : +- BroadcastExchange (8) + : : : +- * Project (7) + : : : +- * Filter (6) + : : : +- * ColumnarToRow (5) + : : : +- Scan parquet default.date_dim (4) + : : +- BroadcastExchange (14) + : : +- * Filter (13) + : : +- * ColumnarToRow (12) + : : +- Scan parquet default.store (11) + : +- * Sort (23) + : +- Exchange (22) + : +- * Filter (21) + : +- * ColumnarToRow (20) + : +- Scan parquet default.item (19) +- * Sort (51) +- Exchange (50) +- * Project (49) - +- * BroadcastHashJoin Inner BuildRight (48) - :- * Filter (46) - : +- * ColumnarToRow (45) - : +- Scan parquet default.catalog_sales (44) - +- ReusedExchange (47) + +- * SortMergeJoin Inner (48) + :- * Sort (39) + : +- Exchange (38) + : +- * Project (37) + : +- * BroadcastHashJoin Inner BuildRight (36) + : :- * Filter (30) + : : +- * ColumnarToRow (29) + : : +- Scan parquet default.store_returns (28) + : +- BroadcastExchange (35) + : +- * Project (34) + : +- * Filter (33) + : +- * ColumnarToRow (32) + : +- Scan parquet default.date_dim (31) + +- * Sort (47) + +- Exchange (46) + +- * Project (45) + +- * BroadcastHashJoin Inner BuildRight (44) + :- * Filter (42) + : +- * ColumnarToRow (41) + : +- Scan parquet default.catalog_sales (40) + +- ReusedExchange (43) (1) Scan parquet default.store_sales @@ -132,7 +132,7 @@ Input [7]: [ss_item_sk#2, ss_customer_sk#3, ss_store_sk#4, ss_ticket_number#5, s (17) Exchange Input [5]: [ss_item_sk#2, ss_customer_sk#3, ss_ticket_number#5, ss_quantity#6, s_state#11] -Arguments: hashpartitioning(ss_item_sk#2, 5), true, [id=#13] +Arguments: hashpartitioning(ss_item_sk#2, 5), ENSURE_REQUIREMENTS, [id=#13] (18) Sort [codegen id : 4] Input [5]: [ss_item_sk#2, ss_customer_sk#3, ss_ticket_number#5, ss_quantity#6, s_state#11] @@ -154,7 +154,7 @@ Condition : isnotnull(i_item_sk#14) (22) Exchange Input [3]: [i_item_sk#14, i_item_id#15, i_item_desc#16] -Arguments: hashpartitioning(i_item_sk#14, 5), true, [id=#17] +Arguments: hashpartitioning(i_item_sk#14, 5), ENSURE_REQUIREMENTS, [id=#17] (23) Sort [codegen id : 6] Input [3]: [i_item_sk#14, i_item_id#15, i_item_desc#16] @@ -171,7 +171,7 @@ Input [8]: [ss_item_sk#2, ss_customer_sk#3, ss_ticket_number#5, ss_quantity#6, s (26) Exchange Input [7]: [ss_item_sk#2, ss_customer_sk#3, ss_ticket_number#5, ss_quantity#6, s_state#11, i_item_id#15, i_item_desc#16] -Arguments: hashpartitioning(cast(ss_customer_sk#3 as bigint), cast(ss_item_sk#2 as bigint), cast(ss_ticket_number#5 as bigint), 5), true, [id=#18] +Arguments: hashpartitioning(cast(ss_customer_sk#3 as bigint), cast(ss_item_sk#2 as bigint), cast(ss_ticket_number#5 as bigint), 5), ENSURE_REQUIREMENTS, [id=#18] (27) Sort [codegen id : 8] Input [7]: [ss_item_sk#2, ss_customer_sk#3, ss_ticket_number#5, ss_quantity#6, s_state#11, i_item_id#15, i_item_desc#16] @@ -224,89 +224,89 @@ Input [6]: [sr_returned_date_sk#19, sr_item_sk#20, sr_customer_sk#21, sr_ticket_ (38) Exchange Input [4]: [sr_item_sk#20, sr_customer_sk#21, sr_ticket_number#22, sr_return_quantity#23] -Arguments: hashpartitioning(sr_customer_sk#21, sr_item_sk#20, sr_ticket_number#22, 5), true, [id=#27] +Arguments: hashpartitioning(sr_customer_sk#21, sr_item_sk#20, 5), ENSURE_REQUIREMENTS, [id=#27] (39) Sort [codegen id : 11] Input [4]: [sr_item_sk#20, sr_customer_sk#21, sr_ticket_number#22, sr_return_quantity#23] -Arguments: [sr_customer_sk#21 ASC NULLS FIRST, sr_item_sk#20 ASC NULLS FIRST, sr_ticket_number#22 ASC NULLS FIRST], false, 0 - -(40) SortMergeJoin [codegen id : 12] -Left keys [3]: [cast(ss_customer_sk#3 as bigint), cast(ss_item_sk#2 as bigint), cast(ss_ticket_number#5 as bigint)] -Right keys [3]: [sr_customer_sk#21, sr_item_sk#20, sr_ticket_number#22] -Join condition: None - -(41) Project [codegen id : 12] -Output [7]: [ss_quantity#6, s_state#11, i_item_id#15, i_item_desc#16, sr_item_sk#20, sr_customer_sk#21, sr_return_quantity#23] -Input [11]: [ss_item_sk#2, ss_customer_sk#3, ss_ticket_number#5, ss_quantity#6, s_state#11, i_item_id#15, i_item_desc#16, sr_item_sk#20, sr_customer_sk#21, sr_ticket_number#22, sr_return_quantity#23] - -(42) Exchange -Input [7]: [ss_quantity#6, s_state#11, i_item_id#15, i_item_desc#16, sr_item_sk#20, sr_customer_sk#21, sr_return_quantity#23] -Arguments: hashpartitioning(sr_customer_sk#21, sr_item_sk#20, 5), true, [id=#28] - -(43) Sort [codegen id : 13] -Input [7]: [ss_quantity#6, s_state#11, i_item_id#15, i_item_desc#16, sr_item_sk#20, sr_customer_sk#21, sr_return_quantity#23] Arguments: [sr_customer_sk#21 ASC NULLS FIRST, sr_item_sk#20 ASC NULLS FIRST], false, 0 -(44) Scan parquet default.catalog_sales -Output [4]: [cs_sold_date_sk#29, cs_bill_customer_sk#30, cs_item_sk#31, cs_quantity#32] +(40) Scan parquet default.catalog_sales +Output [4]: [cs_sold_date_sk#28, cs_bill_customer_sk#29, cs_item_sk#30, cs_quantity#31] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_sales] PushedFilters: [IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk), IsNotNull(cs_sold_date_sk)] ReadSchema: struct -(45) ColumnarToRow [codegen id : 15] -Input [4]: [cs_sold_date_sk#29, cs_bill_customer_sk#30, cs_item_sk#31, cs_quantity#32] +(41) ColumnarToRow [codegen id : 13] +Input [4]: [cs_sold_date_sk#28, cs_bill_customer_sk#29, cs_item_sk#30, cs_quantity#31] + +(42) Filter [codegen id : 13] +Input [4]: [cs_sold_date_sk#28, cs_bill_customer_sk#29, cs_item_sk#30, cs_quantity#31] +Condition : ((isnotnull(cs_bill_customer_sk#29) AND isnotnull(cs_item_sk#30)) AND isnotnull(cs_sold_date_sk#28)) -(46) Filter [codegen id : 15] -Input [4]: [cs_sold_date_sk#29, cs_bill_customer_sk#30, cs_item_sk#31, cs_quantity#32] -Condition : ((isnotnull(cs_bill_customer_sk#30) AND isnotnull(cs_item_sk#31)) AND isnotnull(cs_sold_date_sk#29)) +(43) ReusedExchange [Reuses operator id: 35] +Output [1]: [d_date_sk#32] -(47) ReusedExchange [Reuses operator id: 35] -Output [1]: [d_date_sk#33] +(44) BroadcastHashJoin [codegen id : 13] +Left keys [1]: [cs_sold_date_sk#28] +Right keys [1]: [d_date_sk#32] +Join condition: None + +(45) Project [codegen id : 13] +Output [3]: [cs_bill_customer_sk#29, cs_item_sk#30, cs_quantity#31] +Input [5]: [cs_sold_date_sk#28, cs_bill_customer_sk#29, cs_item_sk#30, cs_quantity#31, d_date_sk#32] + +(46) Exchange +Input [3]: [cs_bill_customer_sk#29, cs_item_sk#30, cs_quantity#31] +Arguments: hashpartitioning(cast(cs_bill_customer_sk#29 as bigint), cast(cs_item_sk#30 as bigint), 5), ENSURE_REQUIREMENTS, [id=#33] -(48) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [cs_sold_date_sk#29] -Right keys [1]: [d_date_sk#33] +(47) Sort [codegen id : 14] +Input [3]: [cs_bill_customer_sk#29, cs_item_sk#30, cs_quantity#31] +Arguments: [cast(cs_bill_customer_sk#29 as bigint) ASC NULLS FIRST, cast(cs_item_sk#30 as bigint) ASC NULLS FIRST], false, 0 + +(48) SortMergeJoin [codegen id : 15] +Left keys [2]: [sr_customer_sk#21, sr_item_sk#20] +Right keys [2]: [cast(cs_bill_customer_sk#29 as bigint), cast(cs_item_sk#30 as bigint)] Join condition: None (49) Project [codegen id : 15] -Output [3]: [cs_bill_customer_sk#30, cs_item_sk#31, cs_quantity#32] -Input [5]: [cs_sold_date_sk#29, cs_bill_customer_sk#30, cs_item_sk#31, cs_quantity#32, d_date_sk#33] +Output [5]: [sr_item_sk#20, sr_customer_sk#21, sr_ticket_number#22, sr_return_quantity#23, cs_quantity#31] +Input [7]: [sr_item_sk#20, sr_customer_sk#21, sr_ticket_number#22, sr_return_quantity#23, cs_bill_customer_sk#29, cs_item_sk#30, cs_quantity#31] (50) Exchange -Input [3]: [cs_bill_customer_sk#30, cs_item_sk#31, cs_quantity#32] -Arguments: hashpartitioning(cast(cs_bill_customer_sk#30 as bigint), cast(cs_item_sk#31 as bigint), 5), true, [id=#34] +Input [5]: [sr_item_sk#20, sr_customer_sk#21, sr_ticket_number#22, sr_return_quantity#23, cs_quantity#31] +Arguments: hashpartitioning(sr_customer_sk#21, sr_item_sk#20, sr_ticket_number#22, 5), ENSURE_REQUIREMENTS, [id=#34] (51) Sort [codegen id : 16] -Input [3]: [cs_bill_customer_sk#30, cs_item_sk#31, cs_quantity#32] -Arguments: [cast(cs_bill_customer_sk#30 as bigint) ASC NULLS FIRST, cast(cs_item_sk#31 as bigint) ASC NULLS FIRST], false, 0 +Input [5]: [sr_item_sk#20, sr_customer_sk#21, sr_ticket_number#22, sr_return_quantity#23, cs_quantity#31] +Arguments: [sr_customer_sk#21 ASC NULLS FIRST, sr_item_sk#20 ASC NULLS FIRST, sr_ticket_number#22 ASC NULLS FIRST], false, 0 (52) SortMergeJoin [codegen id : 17] -Left keys [2]: [sr_customer_sk#21, sr_item_sk#20] -Right keys [2]: [cast(cs_bill_customer_sk#30 as bigint), cast(cs_item_sk#31 as bigint)] +Left keys [3]: [cast(ss_customer_sk#3 as bigint), cast(ss_item_sk#2 as bigint), cast(ss_ticket_number#5 as bigint)] +Right keys [3]: [sr_customer_sk#21, sr_item_sk#20, sr_ticket_number#22] Join condition: None (53) Project [codegen id : 17] -Output [6]: [ss_quantity#6, sr_return_quantity#23, cs_quantity#32, s_state#11, i_item_id#15, i_item_desc#16] -Input [10]: [ss_quantity#6, s_state#11, i_item_id#15, i_item_desc#16, sr_item_sk#20, sr_customer_sk#21, sr_return_quantity#23, cs_bill_customer_sk#30, cs_item_sk#31, cs_quantity#32] +Output [6]: [ss_quantity#6, sr_return_quantity#23, cs_quantity#31, s_state#11, i_item_id#15, i_item_desc#16] +Input [12]: [ss_item_sk#2, ss_customer_sk#3, ss_ticket_number#5, ss_quantity#6, s_state#11, i_item_id#15, i_item_desc#16, sr_item_sk#20, sr_customer_sk#21, sr_ticket_number#22, sr_return_quantity#23, cs_quantity#31] (54) HashAggregate [codegen id : 17] -Input [6]: [ss_quantity#6, sr_return_quantity#23, cs_quantity#32, s_state#11, i_item_id#15, i_item_desc#16] +Input [6]: [ss_quantity#6, sr_return_quantity#23, cs_quantity#31, s_state#11, i_item_id#15, i_item_desc#16] Keys [3]: [i_item_id#15, i_item_desc#16, s_state#11] -Functions [9]: [partial_count(ss_quantity#6), partial_avg(cast(ss_quantity#6 as bigint)), partial_stddev_samp(cast(ss_quantity#6 as double)), partial_count(sr_return_quantity#23), partial_avg(cast(sr_return_quantity#23 as bigint)), partial_stddev_samp(cast(sr_return_quantity#23 as double)), partial_count(cs_quantity#32), partial_avg(cast(cs_quantity#32 as bigint)), partial_stddev_samp(cast(cs_quantity#32 as double))] +Functions [9]: [partial_count(ss_quantity#6), partial_avg(cast(ss_quantity#6 as bigint)), partial_stddev_samp(cast(ss_quantity#6 as double)), partial_count(sr_return_quantity#23), partial_avg(cast(sr_return_quantity#23 as bigint)), partial_stddev_samp(cast(sr_return_quantity#23 as double)), partial_count(cs_quantity#31), partial_avg(cast(cs_quantity#31 as bigint)), partial_stddev_samp(cast(cs_quantity#31 as double))] Aggregate Attributes [18]: [count#35, sum#36, count#37, n#38, avg#39, m2#40, count#41, sum#42, count#43, n#44, avg#45, m2#46, count#47, sum#48, count#49, n#50, avg#51, m2#52] Results [21]: [i_item_id#15, i_item_desc#16, s_state#11, count#53, sum#54, count#55, n#56, avg#57, m2#58, count#59, sum#60, count#61, n#62, avg#63, m2#64, count#65, sum#66, count#67, n#68, avg#69, m2#70] (55) Exchange Input [21]: [i_item_id#15, i_item_desc#16, s_state#11, count#53, sum#54, count#55, n#56, avg#57, m2#58, count#59, sum#60, count#61, n#62, avg#63, m2#64, count#65, sum#66, count#67, n#68, avg#69, m2#70] -Arguments: hashpartitioning(i_item_id#15, i_item_desc#16, s_state#11, 5), true, [id=#71] +Arguments: hashpartitioning(i_item_id#15, i_item_desc#16, s_state#11, 5), ENSURE_REQUIREMENTS, [id=#71] (56) HashAggregate [codegen id : 18] Input [21]: [i_item_id#15, i_item_desc#16, s_state#11, count#53, sum#54, count#55, n#56, avg#57, m2#58, count#59, sum#60, count#61, n#62, avg#63, m2#64, count#65, sum#66, count#67, n#68, avg#69, m2#70] Keys [3]: [i_item_id#15, i_item_desc#16, s_state#11] -Functions [9]: [count(ss_quantity#6), avg(cast(ss_quantity#6 as bigint)), stddev_samp(cast(ss_quantity#6 as double)), count(sr_return_quantity#23), avg(cast(sr_return_quantity#23 as bigint)), stddev_samp(cast(sr_return_quantity#23 as double)), count(cs_quantity#32), avg(cast(cs_quantity#32 as bigint)), stddev_samp(cast(cs_quantity#32 as double))] -Aggregate Attributes [9]: [count(ss_quantity#6)#72, avg(cast(ss_quantity#6 as bigint))#73, stddev_samp(cast(ss_quantity#6 as double))#74, count(sr_return_quantity#23)#75, avg(cast(sr_return_quantity#23 as bigint))#76, stddev_samp(cast(sr_return_quantity#23 as double))#77, count(cs_quantity#32)#78, avg(cast(cs_quantity#32 as bigint))#79, stddev_samp(cast(cs_quantity#32 as double))#80] -Results [15]: [i_item_id#15, i_item_desc#16, s_state#11, count(ss_quantity#6)#72 AS store_sales_quantitycount#81, avg(cast(ss_quantity#6 as bigint))#73 AS store_sales_quantityave#82, stddev_samp(cast(ss_quantity#6 as double))#74 AS store_sales_quantitystdev#83, (stddev_samp(cast(ss_quantity#6 as double))#74 / avg(cast(ss_quantity#6 as bigint))#73) AS store_sales_quantitycov#84, count(sr_return_quantity#23)#75 AS as_store_returns_quantitycount#85, avg(cast(sr_return_quantity#23 as bigint))#76 AS as_store_returns_quantityave#86, stddev_samp(cast(sr_return_quantity#23 as double))#77 AS as_store_returns_quantitystdev#87, (stddev_samp(cast(sr_return_quantity#23 as double))#77 / avg(cast(sr_return_quantity#23 as bigint))#76) AS store_returns_quantitycov#88, count(cs_quantity#32)#78 AS catalog_sales_quantitycount#89, avg(cast(cs_quantity#32 as bigint))#79 AS catalog_sales_quantityave#90, (stddev_samp(cast(cs_quantity#32 as double))#80 / avg(cast(cs_quantity#32 as bigint))#79) AS catalog_sales_quantitystdev#91, (stddev_samp(cast(cs_quantity#32 as double))#80 / avg(cast(cs_quantity#32 as bigint))#79) AS catalog_sales_quantitycov#92] +Functions [9]: [count(ss_quantity#6), avg(cast(ss_quantity#6 as bigint)), stddev_samp(cast(ss_quantity#6 as double)), count(sr_return_quantity#23), avg(cast(sr_return_quantity#23 as bigint)), stddev_samp(cast(sr_return_quantity#23 as double)), count(cs_quantity#31), avg(cast(cs_quantity#31 as bigint)), stddev_samp(cast(cs_quantity#31 as double))] +Aggregate Attributes [9]: [count(ss_quantity#6)#72, avg(cast(ss_quantity#6 as bigint))#73, stddev_samp(cast(ss_quantity#6 as double))#74, count(sr_return_quantity#23)#75, avg(cast(sr_return_quantity#23 as bigint))#76, stddev_samp(cast(sr_return_quantity#23 as double))#77, count(cs_quantity#31)#78, avg(cast(cs_quantity#31 as bigint))#79, stddev_samp(cast(cs_quantity#31 as double))#80] +Results [15]: [i_item_id#15, i_item_desc#16, s_state#11, count(ss_quantity#6)#72 AS store_sales_quantitycount#81, avg(cast(ss_quantity#6 as bigint))#73 AS store_sales_quantityave#82, stddev_samp(cast(ss_quantity#6 as double))#74 AS store_sales_quantitystdev#83, (stddev_samp(cast(ss_quantity#6 as double))#74 / avg(cast(ss_quantity#6 as bigint))#73) AS store_sales_quantitycov#84, count(sr_return_quantity#23)#75 AS as_store_returns_quantitycount#85, avg(cast(sr_return_quantity#23 as bigint))#76 AS as_store_returns_quantityave#86, stddev_samp(cast(sr_return_quantity#23 as double))#77 AS as_store_returns_quantitystdev#87, (stddev_samp(cast(sr_return_quantity#23 as double))#77 / avg(cast(sr_return_quantity#23 as bigint))#76) AS store_returns_quantitycov#88, count(cs_quantity#31)#78 AS catalog_sales_quantitycount#89, avg(cast(cs_quantity#31 as bigint))#79 AS catalog_sales_quantityave#90, (stddev_samp(cast(cs_quantity#31 as double))#80 / avg(cast(cs_quantity#31 as bigint))#79) AS catalog_sales_quantitystdev#91, (stddev_samp(cast(cs_quantity#31 as double))#80 / avg(cast(cs_quantity#31 as bigint))#79) AS catalog_sales_quantitycov#92] (57) TakeOrderedAndProject Input [15]: [i_item_id#15, i_item_desc#16, s_state#11, store_sales_quantitycount#81, store_sales_quantityave#82, store_sales_quantitystdev#83, store_sales_quantitycov#84, as_store_returns_quantitycount#85, as_store_returns_quantityave#86, as_store_returns_quantitystdev#87, store_returns_quantitycov#88, catalog_sales_quantitycount#89, catalog_sales_quantityave#90, catalog_sales_quantitystdev#91, catalog_sales_quantitycov#92] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/simplified.txt index bfb59441f483b..79226a34e6768 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/simplified.txt @@ -6,67 +6,67 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_state,store_sales_quantitycount,s WholeStageCodegen (17) HashAggregate [i_item_id,i_item_desc,s_state,ss_quantity,sr_return_quantity,cs_quantity] [count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2] Project [ss_quantity,sr_return_quantity,cs_quantity,s_state,i_item_id,i_item_desc] - SortMergeJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk] + SortMergeJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number] InputAdapter - WholeStageCodegen (13) - Sort [sr_customer_sk,sr_item_sk] + WholeStageCodegen (8) + Sort [ss_customer_sk,ss_item_sk,ss_ticket_number] InputAdapter - Exchange [sr_customer_sk,sr_item_sk] #2 - WholeStageCodegen (12) - Project [ss_quantity,s_state,i_item_id,i_item_desc,sr_item_sk,sr_customer_sk,sr_return_quantity] - SortMergeJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number] + Exchange [ss_customer_sk,ss_item_sk,ss_ticket_number] #2 + WholeStageCodegen (7) + Project [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,s_state,i_item_id,i_item_desc] + SortMergeJoin [ss_item_sk,i_item_sk] InputAdapter - WholeStageCodegen (8) - Sort [ss_customer_sk,ss_item_sk,ss_ticket_number] + WholeStageCodegen (4) + Sort [ss_item_sk] InputAdapter - Exchange [ss_customer_sk,ss_item_sk,ss_ticket_number] #3 - WholeStageCodegen (7) - Project [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,s_state,i_item_id,i_item_desc] - SortMergeJoin [ss_item_sk,i_item_sk] - InputAdapter - WholeStageCodegen (4) - Sort [ss_item_sk] - InputAdapter - Exchange [ss_item_sk] #4 - WholeStageCodegen (3) - Project [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,s_state] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_customer_sk,ss_item_sk,ss_ticket_number,ss_sold_date_sk,ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_quarter_name,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_quarter_name] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (2) - Filter [s_store_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store [s_store_sk,s_state] - InputAdapter - WholeStageCodegen (6) - Sort [i_item_sk] - InputAdapter - Exchange [i_item_sk] #7 - WholeStageCodegen (5) - Filter [i_item_sk] + Exchange [ss_item_sk] #3 + WholeStageCodegen (3) + Project [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,s_state] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_customer_sk,ss_item_sk,ss_ticket_number,ss_sold_date_sk,ss_store_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (1) + Project [d_date_sk] + Filter [d_quarter_name,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_item_sk,i_item_id,i_item_desc] + Scan parquet default.date_dim [d_date_sk,d_quarter_name] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (2) + Filter [s_store_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store [s_store_sk,s_state] + InputAdapter + WholeStageCodegen (6) + Sort [i_item_sk] + InputAdapter + Exchange [i_item_sk] #6 + WholeStageCodegen (5) + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_item_id,i_item_desc] + InputAdapter + WholeStageCodegen (16) + Sort [sr_customer_sk,sr_item_sk,sr_ticket_number] + InputAdapter + Exchange [sr_customer_sk,sr_item_sk,sr_ticket_number] #7 + WholeStageCodegen (15) + Project [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,cs_quantity] + SortMergeJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk] InputAdapter WholeStageCodegen (11) - Sort [sr_customer_sk,sr_item_sk,sr_ticket_number] + Sort [sr_customer_sk,sr_item_sk] InputAdapter - Exchange [sr_customer_sk,sr_item_sk,sr_ticket_number] #8 + Exchange [sr_customer_sk,sr_item_sk] #8 WholeStageCodegen (10) Project [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity] BroadcastHashJoin [sr_returned_date_sk,d_date_sk] @@ -82,17 +82,17 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_state,store_sales_quantitycount,s ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_quarter_name] - InputAdapter - WholeStageCodegen (16) - Sort [cs_bill_customer_sk,cs_item_sk] - InputAdapter - Exchange [cs_bill_customer_sk,cs_item_sk] #10 - WholeStageCodegen (15) - Project [cs_bill_customer_sk,cs_item_sk,cs_quantity] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk,cs_quantity] InputAdapter - ReusedExchange [d_date_sk] #9 + WholeStageCodegen (14) + Sort [cs_bill_customer_sk,cs_item_sk] + InputAdapter + Exchange [cs_bill_customer_sk,cs_item_sk] #10 + WholeStageCodegen (13) + Project [cs_bill_customer_sk,cs_item_sk,cs_quantity] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Filter [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk,cs_quantity] + InputAdapter + ReusedExchange [d_date_sk] #9 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 516f782057631..12e95ba50cd0d 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 @@ -34,24 +34,24 @@ TakeOrderedAndProject (53) +- * Sort (46) +- Exchange (45) +- * Project (44) - +- * SortMergeJoin Inner (43) - :- * Sort (37) - : +- Exchange (36) - : +- * Project (35) - : +- * BroadcastHashJoin Inner BuildRight (34) - : :- * Project (29) - : : +- * Filter (28) - : : +- * ColumnarToRow (27) - : : +- Scan parquet default.customer (26) - : +- BroadcastExchange (33) - : +- * Filter (32) - : +- * ColumnarToRow (31) - : +- Scan parquet default.customer_address (30) - +- * Sort (42) - +- Exchange (41) - +- * Filter (40) - +- * ColumnarToRow (39) - +- Scan parquet default.customer_demographics (38) + +- * BroadcastHashJoin Inner BuildRight (43) + :- * Project (38) + : +- * SortMergeJoin Inner (37) + : :- * Sort (31) + : : +- Exchange (30) + : : +- * Project (29) + : : +- * Filter (28) + : : +- * ColumnarToRow (27) + : : +- Scan parquet default.customer (26) + : +- * Sort (36) + : +- Exchange (35) + : +- * Filter (34) + : +- * ColumnarToRow (33) + : +- Scan parquet default.customer_demographics (32) + +- BroadcastExchange (42) + +- * Filter (41) + +- * ColumnarToRow (40) + +- Scan parquet default.customer_address (39) (1) Scan parquet default.catalog_sales @@ -159,7 +159,7 @@ Input [10]: [cs_bill_customer_sk#2, cs_item_sk#4, cs_quantity#5, cs_list_price#6 (24) Exchange Input [8]: [cs_bill_customer_sk#2, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, i_item_id#19] -Arguments: hashpartitioning(cs_bill_customer_sk#2, 5), true, [id=#21] +Arguments: hashpartitioning(cs_bill_customer_sk#2, 5), ENSURE_REQUIREMENTS, [id=#21] (25) Sort [codegen id : 5] Input [8]: [cs_bill_customer_sk#2, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, i_item_id#19] @@ -172,89 +172,89 @@ Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [In(c_birth_month, [1,6,8,9,12,2]), IsNotNull(c_customer_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct -(27) ColumnarToRow [codegen id : 7] +(27) ColumnarToRow [codegen id : 6] Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_month#25, c_birth_year#26] -(28) Filter [codegen id : 7] +(28) Filter [codegen id : 6] Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_month#25, c_birth_year#26] Condition : (((c_birth_month#25 IN (1,6,8,9,12,2) AND isnotnull(c_customer_sk#22)) AND isnotnull(c_current_cdemo_sk#23)) AND isnotnull(c_current_addr_sk#24)) -(29) Project [codegen id : 7] +(29) Project [codegen id : 6] Output [4]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_year#26] Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_month#25, c_birth_year#26] -(30) Scan parquet default.customer_address -Output [4]: [ca_address_sk#27, ca_county#28, ca_state#29, ca_country#30] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [In(ca_state, [MS,IN,ND,OK,NM,VA]), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(31) ColumnarToRow [codegen id : 6] -Input [4]: [ca_address_sk#27, ca_county#28, ca_state#29, ca_country#30] - -(32) Filter [codegen id : 6] -Input [4]: [ca_address_sk#27, ca_county#28, ca_state#29, ca_country#30] -Condition : (ca_state#29 IN (MS,IN,ND,OK,NM,VA) AND isnotnull(ca_address_sk#27)) - -(33) BroadcastExchange -Input [4]: [ca_address_sk#27, ca_county#28, ca_state#29, ca_country#30] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#31] - -(34) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [c_current_addr_sk#24] -Right keys [1]: [ca_address_sk#27] -Join condition: None - -(35) Project [codegen id : 7] -Output [6]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26, ca_county#28, ca_state#29, ca_country#30] -Input [8]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_year#26, ca_address_sk#27, ca_county#28, ca_state#29, ca_country#30] - -(36) Exchange -Input [6]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26, ca_county#28, ca_state#29, ca_country#30] -Arguments: hashpartitioning(c_current_cdemo_sk#23, 5), true, [id=#32] +(30) Exchange +Input [4]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_year#26] +Arguments: hashpartitioning(c_current_cdemo_sk#23, 5), ENSURE_REQUIREMENTS, [id=#27] -(37) Sort [codegen id : 8] -Input [6]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26, ca_county#28, ca_state#29, ca_country#30] +(31) Sort [codegen id : 7] +Input [4]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_year#26] Arguments: [c_current_cdemo_sk#23 ASC NULLS FIRST], false, 0 -(38) Scan parquet default.customer_demographics -Output [1]: [cd_demo_sk#33] +(32) Scan parquet default.customer_demographics +Output [1]: [cd_demo_sk#28] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(39) ColumnarToRow [codegen id : 9] -Input [1]: [cd_demo_sk#33] +(33) ColumnarToRow [codegen id : 8] +Input [1]: [cd_demo_sk#28] -(40) Filter [codegen id : 9] -Input [1]: [cd_demo_sk#33] -Condition : isnotnull(cd_demo_sk#33) +(34) Filter [codegen id : 8] +Input [1]: [cd_demo_sk#28] +Condition : isnotnull(cd_demo_sk#28) -(41) Exchange -Input [1]: [cd_demo_sk#33] -Arguments: hashpartitioning(cd_demo_sk#33, 5), true, [id=#34] +(35) Exchange +Input [1]: [cd_demo_sk#28] +Arguments: hashpartitioning(cd_demo_sk#28, 5), ENSURE_REQUIREMENTS, [id=#29] -(42) Sort [codegen id : 10] -Input [1]: [cd_demo_sk#33] -Arguments: [cd_demo_sk#33 ASC NULLS FIRST], false, 0 +(36) Sort [codegen id : 9] +Input [1]: [cd_demo_sk#28] +Arguments: [cd_demo_sk#28 ASC NULLS FIRST], false, 0 -(43) SortMergeJoin [codegen id : 11] +(37) SortMergeJoin [codegen id : 11] Left keys [1]: [c_current_cdemo_sk#23] -Right keys [1]: [cd_demo_sk#33] +Right keys [1]: [cd_demo_sk#28] +Join condition: None + +(38) Project [codegen id : 11] +Output [3]: [c_customer_sk#22, c_current_addr_sk#24, c_birth_year#26] +Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_year#26, cd_demo_sk#28] + +(39) Scan parquet default.customer_address +Output [4]: [ca_address_sk#30, ca_county#31, ca_state#32, ca_country#33] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer_address] +PushedFilters: [In(ca_state, [MS,IN,ND,OK,NM,VA]), IsNotNull(ca_address_sk)] +ReadSchema: struct + +(40) ColumnarToRow [codegen id : 10] +Input [4]: [ca_address_sk#30, ca_county#31, ca_state#32, ca_country#33] + +(41) Filter [codegen id : 10] +Input [4]: [ca_address_sk#30, ca_county#31, ca_state#32, ca_country#33] +Condition : (ca_state#32 IN (MS,IN,ND,OK,NM,VA) AND isnotnull(ca_address_sk#30)) + +(42) BroadcastExchange +Input [4]: [ca_address_sk#30, ca_county#31, ca_state#32, ca_country#33] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#34] + +(43) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [c_current_addr_sk#24] +Right keys [1]: [ca_address_sk#30] Join condition: None (44) Project [codegen id : 11] -Output [5]: [c_customer_sk#22, c_birth_year#26, ca_county#28, ca_state#29, ca_country#30] -Input [7]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26, ca_county#28, ca_state#29, ca_country#30, cd_demo_sk#33] +Output [5]: [c_customer_sk#22, c_birth_year#26, ca_county#31, ca_state#32, ca_country#33] +Input [7]: [c_customer_sk#22, c_current_addr_sk#24, c_birth_year#26, ca_address_sk#30, ca_county#31, ca_state#32, ca_country#33] (45) Exchange -Input [5]: [c_customer_sk#22, c_birth_year#26, ca_county#28, ca_state#29, ca_country#30] -Arguments: hashpartitioning(c_customer_sk#22, 5), true, [id=#35] +Input [5]: [c_customer_sk#22, c_birth_year#26, ca_county#31, ca_state#32, ca_country#33] +Arguments: hashpartitioning(c_customer_sk#22, 5), ENSURE_REQUIREMENTS, [id=#35] (46) Sort [codegen id : 12] -Input [5]: [c_customer_sk#22, c_birth_year#26, ca_county#28, ca_state#29, ca_country#30] +Input [5]: [c_customer_sk#22, c_birth_year#26, ca_county#31, ca_state#32, ca_country#33] Arguments: [c_customer_sk#22 ASC NULLS FIRST], false, 0 (47) SortMergeJoin [codegen id : 13] @@ -263,12 +263,12 @@ Right keys [1]: [c_customer_sk#22] Join condition: None (48) Project [codegen id : 13] -Output [11]: [cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, c_birth_year#26, i_item_id#19, ca_country#30, ca_state#29, ca_county#28] -Input [13]: [cs_bill_customer_sk#2, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, i_item_id#19, c_customer_sk#22, c_birth_year#26, ca_county#28, ca_state#29, ca_country#30] +Output [11]: [cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, c_birth_year#26, i_item_id#19, ca_country#33, ca_state#32, ca_county#31] +Input [13]: [cs_bill_customer_sk#2, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, i_item_id#19, c_customer_sk#22, c_birth_year#26, ca_county#31, ca_state#32, ca_country#33] (49) Expand [codegen id : 13] -Input [11]: [cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, c_birth_year#26, i_item_id#19, ca_country#30, ca_state#29, ca_county#28] -Arguments: [List(cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, c_birth_year#26, i_item_id#19, ca_country#30, ca_state#29, ca_county#28, 0), List(cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, c_birth_year#26, i_item_id#19, ca_country#30, ca_state#29, null, 1), List(cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, c_birth_year#26, i_item_id#19, ca_country#30, null, null, 3), List(cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, c_birth_year#26, i_item_id#19, null, null, null, 7), List(cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, c_birth_year#26, null, null, null, null, 15)], [cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, c_birth_year#26, i_item_id#36, ca_country#37, ca_state#38, ca_county#39, spark_grouping_id#40] +Input [11]: [cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, c_birth_year#26, i_item_id#19, ca_country#33, ca_state#32, ca_county#31] +Arguments: [List(cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, c_birth_year#26, i_item_id#19, ca_country#33, ca_state#32, ca_county#31, 0), List(cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, c_birth_year#26, i_item_id#19, ca_country#33, ca_state#32, null, 1), List(cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, c_birth_year#26, i_item_id#19, ca_country#33, null, null, 3), List(cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, c_birth_year#26, i_item_id#19, null, null, null, 7), List(cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, c_birth_year#26, null, null, null, null, 15)], [cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, c_birth_year#26, i_item_id#36, ca_country#37, ca_state#38, ca_county#39, spark_grouping_id#40] (50) HashAggregate [codegen id : 13] Input [12]: [cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, c_birth_year#26, i_item_id#36, ca_country#37, ca_state#38, ca_county#39, spark_grouping_id#40] @@ -279,7 +279,7 @@ Results [19]: [i_item_id#36, ca_country#37, ca_state#38, ca_county#39, spark_gro (51) Exchange Input [19]: [i_item_id#36, ca_country#37, ca_state#38, ca_county#39, spark_grouping_id#40, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62, sum#63, count#64, sum#65, count#66, sum#67, count#68] -Arguments: hashpartitioning(i_item_id#36, ca_country#37, ca_state#38, ca_county#39, spark_grouping_id#40, 5), true, [id=#69] +Arguments: hashpartitioning(i_item_id#36, ca_country#37, ca_state#38, ca_county#39, spark_grouping_id#40, 5), ENSURE_REQUIREMENTS, [id=#69] (52) HashAggregate [codegen id : 14] Input [19]: [i_item_id#36, ca_country#37, ca_state#38, ca_county#39, spark_grouping_id#40, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62, sum#63, count#64, sum#65, count#66, sum#67, count#68] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.sf100/simplified.txt index 8c76e7cab3310..8069d43c3451a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.sf100/simplified.txt @@ -54,34 +54,34 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag Exchange [c_customer_sk] #6 WholeStageCodegen (11) Project [c_customer_sk,c_birth_year,ca_county,ca_state,ca_country] - SortMergeJoin [c_current_cdemo_sk,cd_demo_sk] - InputAdapter - WholeStageCodegen (8) - Sort [c_current_cdemo_sk] - InputAdapter - Exchange [c_current_cdemo_sk] #7 - WholeStageCodegen (7) - Project [c_customer_sk,c_current_cdemo_sk,c_birth_year,ca_county,ca_state,ca_country] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [c_customer_sk,c_current_addr_sk,c_birth_year] + SortMergeJoin [c_current_cdemo_sk,cd_demo_sk] + InputAdapter + WholeStageCodegen (7) + Sort [c_current_cdemo_sk] + InputAdapter + Exchange [c_current_cdemo_sk] #7 + WholeStageCodegen (6) Project [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] Filter [c_birth_month,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] ColumnarToRow InputAdapter Scan parquet default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (6) - Filter [ca_state,ca_address_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] + InputAdapter + WholeStageCodegen (9) + Sort [cd_demo_sk] + InputAdapter + Exchange [cd_demo_sk] #8 + WholeStageCodegen (8) + Filter [cd_demo_sk] + ColumnarToRow + InputAdapter + Scan parquet default.customer_demographics [cd_demo_sk] InputAdapter - WholeStageCodegen (10) - Sort [cd_demo_sk] - InputAdapter - Exchange [cd_demo_sk] #9 - WholeStageCodegen (9) - Filter [cd_demo_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer_demographics [cd_demo_sk] + BroadcastExchange #9 + WholeStageCodegen (10) + Filter [ca_state,ca_address_sk] + ColumnarToRow + InputAdapter + Scan parquet default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] 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 88b5168f6049c..4627bc19f25f0 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 @@ -4,248 +4,248 @@ TakeOrderedAndProject (45) +- Exchange (43) +- * HashAggregate (42) +- * Project (41) - +- * SortMergeJoin Inner (40) - :- * Sort (25) - : +- Exchange (24) - : +- * Project (23) - : +- * BroadcastHashJoin Inner BuildRight (22) - : :- * Project (17) - : : +- * BroadcastHashJoin Inner BuildRight (16) - : : :- * Project (10) - : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.store_sales (1) - : : : +- BroadcastExchange (8) - : : : +- * Project (7) - : : : +- * Filter (6) - : : : +- * ColumnarToRow (5) - : : : +- Scan parquet default.item (4) - : : +- BroadcastExchange (15) - : : +- * Project (14) - : : +- * Filter (13) - : : +- * ColumnarToRow (12) - : : +- Scan parquet default.date_dim (11) - : +- BroadcastExchange (21) - : +- * Filter (20) - : +- * ColumnarToRow (19) - : +- Scan parquet default.store (18) - +- * Sort (39) - +- Exchange (38) - +- * Project (37) - +- * SortMergeJoin Inner (36) - :- * Sort (30) - : +- Exchange (29) - : +- * Filter (28) - : +- * ColumnarToRow (27) - : +- Scan parquet default.customer (26) - +- * Sort (35) - +- Exchange (34) - +- * Filter (33) - +- * ColumnarToRow (32) - +- Scan parquet default.customer_address (31) - - -(1) Scan parquet default.store_sales -Output [5]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_store_sk#4, ss_ext_sales_price#5] + +- * BroadcastHashJoin Inner BuildRight (40) + :- * Project (34) + : +- * SortMergeJoin Inner (33) + : :- * Sort (18) + : : +- Exchange (17) + : : +- * Project (16) + : : +- * BroadcastHashJoin Inner BuildRight (15) + : : :- * Project (10) + : : : +- * BroadcastHashJoin Inner BuildLeft (9) + : : : :- BroadcastExchange (5) + : : : : +- * Project (4) + : : : : +- * Filter (3) + : : : : +- * ColumnarToRow (2) + : : : : +- Scan parquet default.date_dim (1) + : : : +- * Filter (8) + : : : +- * ColumnarToRow (7) + : : : +- Scan parquet default.store_sales (6) + : : +- BroadcastExchange (14) + : : +- * Filter (13) + : : +- * ColumnarToRow (12) + : : +- Scan parquet default.store (11) + : +- * Sort (32) + : +- Exchange (31) + : +- * Project (30) + : +- * SortMergeJoin Inner (29) + : :- * Sort (23) + : : +- Exchange (22) + : : +- * Filter (21) + : : +- * ColumnarToRow (20) + : : +- Scan parquet default.customer (19) + : +- * Sort (28) + : +- Exchange (27) + : +- * Filter (26) + : +- * ColumnarToRow (25) + : +- Scan parquet default.customer_address (24) + +- BroadcastExchange (39) + +- * Project (38) + +- * Filter (37) + +- * ColumnarToRow (36) + +- Scan parquet default.item (35) + + +(1) Scan parquet default.date_dim +Output [3]: [d_date_sk#1, d_year#2, d_moy#3] Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_sold_date_sk), IsNotNull(ss_item_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 4] -Input [5]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_store_sk#4, ss_ext_sales_price#5] - -(3) Filter [codegen id : 4] -Input [5]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_store_sk#4, ss_ext_sales_price#5] -Condition : (((isnotnull(ss_sold_date_sk#1) AND isnotnull(ss_item_sk#2)) AND isnotnull(ss_customer_sk#3)) AND isnotnull(ss_store_sk#4)) - -(4) Scan parquet default.item -Output [6]: [i_item_sk#6, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10, i_manager_id#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manager_id), EqualTo(i_manager_id,8), IsNotNull(i_item_sk)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 1] -Input [6]: [i_item_sk#6, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10, i_manager_id#11] - -(6) Filter [codegen id : 1] -Input [6]: [i_item_sk#6, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10, i_manager_id#11] -Condition : ((isnotnull(i_manager_id#11) AND (i_manager_id#11 = 8)) AND isnotnull(i_item_sk#6)) +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,11), EqualTo(d_year,1998), IsNotNull(d_date_sk)] +ReadSchema: struct -(7) Project [codegen id : 1] -Output [5]: [i_item_sk#6, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10] -Input [6]: [i_item_sk#6, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10, i_manager_id#11] +(2) ColumnarToRow [codegen id : 1] +Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -(8) BroadcastExchange -Input [5]: [i_item_sk#6, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#12] +(3) Filter [codegen id : 1] +Input [3]: [d_date_sk#1, d_year#2, d_moy#3] +Condition : ((((isnotnull(d_moy#3) AND isnotnull(d_year#2)) AND (d_moy#3 = 11)) AND (d_year#2 = 1998)) AND isnotnull(d_date_sk#1)) -(9) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_item_sk#2] -Right keys [1]: [i_item_sk#6] -Join condition: None +(4) Project [codegen id : 1] +Output [1]: [d_date_sk#1] +Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -(10) Project [codegen id : 4] -Output [8]: [ss_sold_date_sk#1, ss_customer_sk#3, ss_store_sk#4, ss_ext_sales_price#5, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10] -Input [10]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_store_sk#4, ss_ext_sales_price#5, i_item_sk#6, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10] +(5) BroadcastExchange +Input [1]: [d_date_sk#1] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#4] -(11) Scan parquet default.date_dim -Output [3]: [d_date_sk#13, d_year#14, d_moy#15] +(6) Scan parquet default.store_sales +Output [5]: [ss_sold_date_sk#5, ss_item_sk#6, ss_customer_sk#7, ss_store_sk#8, ss_ext_sales_price#9] Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,11), EqualTo(d_year,1998), IsNotNull(d_date_sk)] -ReadSchema: struct - -(12) ColumnarToRow [codegen id : 2] -Input [3]: [d_date_sk#13, d_year#14, d_moy#15] - -(13) Filter [codegen id : 2] -Input [3]: [d_date_sk#13, d_year#14, d_moy#15] -Condition : ((((isnotnull(d_moy#15) AND isnotnull(d_year#14)) AND (d_moy#15 = 11)) AND (d_year#14 = 1998)) AND isnotnull(d_date_sk#13)) +Location [not included in comparison]/{warehouse_dir}/store_sales] +PushedFilters: [IsNotNull(ss_sold_date_sk), IsNotNull(ss_item_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_store_sk)] +ReadSchema: struct -(14) Project [codegen id : 2] -Output [1]: [d_date_sk#13] -Input [3]: [d_date_sk#13, d_year#14, d_moy#15] +(7) ColumnarToRow +Input [5]: [ss_sold_date_sk#5, ss_item_sk#6, ss_customer_sk#7, ss_store_sk#8, ss_ext_sales_price#9] -(15) BroadcastExchange -Input [1]: [d_date_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#16] +(8) Filter +Input [5]: [ss_sold_date_sk#5, ss_item_sk#6, ss_customer_sk#7, ss_store_sk#8, ss_ext_sales_price#9] +Condition : (((isnotnull(ss_sold_date_sk#5) AND isnotnull(ss_item_sk#6)) AND isnotnull(ss_customer_sk#7)) AND isnotnull(ss_store_sk#8)) -(16) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#1] -Right keys [1]: [d_date_sk#13] +(9) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [d_date_sk#1] +Right keys [1]: [ss_sold_date_sk#5] Join condition: None -(17) Project [codegen id : 4] -Output [7]: [ss_customer_sk#3, ss_store_sk#4, ss_ext_sales_price#5, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10] -Input [9]: [ss_sold_date_sk#1, ss_customer_sk#3, ss_store_sk#4, ss_ext_sales_price#5, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10, d_date_sk#13] +(10) Project [codegen id : 3] +Output [4]: [ss_item_sk#6, ss_customer_sk#7, ss_store_sk#8, ss_ext_sales_price#9] +Input [6]: [d_date_sk#1, ss_sold_date_sk#5, ss_item_sk#6, ss_customer_sk#7, ss_store_sk#8, ss_ext_sales_price#9] -(18) Scan parquet default.store -Output [2]: [s_store_sk#17, s_zip#18] +(11) Scan parquet default.store +Output [2]: [s_store_sk#10, s_zip#11] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_zip), IsNotNull(s_store_sk)] ReadSchema: struct -(19) ColumnarToRow [codegen id : 3] -Input [2]: [s_store_sk#17, s_zip#18] +(12) ColumnarToRow [codegen id : 2] +Input [2]: [s_store_sk#10, s_zip#11] -(20) Filter [codegen id : 3] -Input [2]: [s_store_sk#17, s_zip#18] -Condition : (isnotnull(s_zip#18) AND isnotnull(s_store_sk#17)) +(13) Filter [codegen id : 2] +Input [2]: [s_store_sk#10, s_zip#11] +Condition : (isnotnull(s_zip#11) AND isnotnull(s_store_sk#10)) -(21) BroadcastExchange -Input [2]: [s_store_sk#17, s_zip#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#19] +(14) BroadcastExchange +Input [2]: [s_store_sk#10, s_zip#11] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#12] -(22) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#4] -Right keys [1]: [s_store_sk#17] +(15) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [ss_store_sk#8] +Right keys [1]: [s_store_sk#10] Join condition: None -(23) Project [codegen id : 4] -Output [7]: [ss_customer_sk#3, ss_ext_sales_price#5, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10, s_zip#18] -Input [9]: [ss_customer_sk#3, ss_store_sk#4, ss_ext_sales_price#5, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10, s_store_sk#17, s_zip#18] +(16) Project [codegen id : 3] +Output [4]: [ss_item_sk#6, ss_customer_sk#7, ss_ext_sales_price#9, s_zip#11] +Input [6]: [ss_item_sk#6, ss_customer_sk#7, ss_store_sk#8, ss_ext_sales_price#9, s_store_sk#10, s_zip#11] -(24) Exchange -Input [7]: [ss_customer_sk#3, ss_ext_sales_price#5, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10, s_zip#18] -Arguments: hashpartitioning(ss_customer_sk#3, 5), true, [id=#20] +(17) Exchange +Input [4]: [ss_item_sk#6, ss_customer_sk#7, ss_ext_sales_price#9, s_zip#11] +Arguments: hashpartitioning(ss_customer_sk#7, 5), ENSURE_REQUIREMENTS, [id=#13] -(25) Sort [codegen id : 5] -Input [7]: [ss_customer_sk#3, ss_ext_sales_price#5, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10, s_zip#18] -Arguments: [ss_customer_sk#3 ASC NULLS FIRST], false, 0 +(18) Sort [codegen id : 4] +Input [4]: [ss_item_sk#6, ss_customer_sk#7, ss_ext_sales_price#9, s_zip#11] +Arguments: [ss_customer_sk#7 ASC NULLS FIRST], false, 0 -(26) Scan parquet default.customer -Output [2]: [c_customer_sk#21, c_current_addr_sk#22] +(19) Scan parquet default.customer +Output [2]: [c_customer_sk#14, c_current_addr_sk#15] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct -(27) ColumnarToRow [codegen id : 6] -Input [2]: [c_customer_sk#21, c_current_addr_sk#22] +(20) ColumnarToRow [codegen id : 5] +Input [2]: [c_customer_sk#14, c_current_addr_sk#15] -(28) Filter [codegen id : 6] -Input [2]: [c_customer_sk#21, c_current_addr_sk#22] -Condition : (isnotnull(c_customer_sk#21) AND isnotnull(c_current_addr_sk#22)) +(21) Filter [codegen id : 5] +Input [2]: [c_customer_sk#14, c_current_addr_sk#15] +Condition : (isnotnull(c_customer_sk#14) AND isnotnull(c_current_addr_sk#15)) -(29) Exchange -Input [2]: [c_customer_sk#21, c_current_addr_sk#22] -Arguments: hashpartitioning(c_current_addr_sk#22, 5), true, [id=#23] +(22) Exchange +Input [2]: [c_customer_sk#14, c_current_addr_sk#15] +Arguments: hashpartitioning(c_current_addr_sk#15, 5), ENSURE_REQUIREMENTS, [id=#16] -(30) Sort [codegen id : 7] -Input [2]: [c_customer_sk#21, c_current_addr_sk#22] -Arguments: [c_current_addr_sk#22 ASC NULLS FIRST], false, 0 +(23) Sort [codegen id : 6] +Input [2]: [c_customer_sk#14, c_current_addr_sk#15] +Arguments: [c_current_addr_sk#15 ASC NULLS FIRST], false, 0 -(31) Scan parquet default.customer_address -Output [2]: [ca_address_sk#24, ca_zip#25] +(24) Scan parquet default.customer_address +Output [2]: [ca_address_sk#17, ca_zip#18] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_zip)] ReadSchema: struct -(32) ColumnarToRow [codegen id : 8] -Input [2]: [ca_address_sk#24, ca_zip#25] +(25) ColumnarToRow [codegen id : 7] +Input [2]: [ca_address_sk#17, ca_zip#18] -(33) Filter [codegen id : 8] -Input [2]: [ca_address_sk#24, ca_zip#25] -Condition : (isnotnull(ca_address_sk#24) AND isnotnull(ca_zip#25)) +(26) Filter [codegen id : 7] +Input [2]: [ca_address_sk#17, ca_zip#18] +Condition : (isnotnull(ca_address_sk#17) AND isnotnull(ca_zip#18)) -(34) Exchange -Input [2]: [ca_address_sk#24, ca_zip#25] -Arguments: hashpartitioning(ca_address_sk#24, 5), true, [id=#26] +(27) Exchange +Input [2]: [ca_address_sk#17, ca_zip#18] +Arguments: hashpartitioning(ca_address_sk#17, 5), ENSURE_REQUIREMENTS, [id=#19] -(35) Sort [codegen id : 9] -Input [2]: [ca_address_sk#24, ca_zip#25] -Arguments: [ca_address_sk#24 ASC NULLS FIRST], false, 0 +(28) Sort [codegen id : 8] +Input [2]: [ca_address_sk#17, ca_zip#18] +Arguments: [ca_address_sk#17 ASC NULLS FIRST], false, 0 -(36) SortMergeJoin [codegen id : 10] -Left keys [1]: [c_current_addr_sk#22] -Right keys [1]: [ca_address_sk#24] +(29) SortMergeJoin [codegen id : 9] +Left keys [1]: [c_current_addr_sk#15] +Right keys [1]: [ca_address_sk#17] Join condition: None -(37) Project [codegen id : 10] -Output [2]: [c_customer_sk#21, ca_zip#25] -Input [4]: [c_customer_sk#21, c_current_addr_sk#22, ca_address_sk#24, ca_zip#25] +(30) Project [codegen id : 9] +Output [2]: [c_customer_sk#14, ca_zip#18] +Input [4]: [c_customer_sk#14, c_current_addr_sk#15, ca_address_sk#17, ca_zip#18] + +(31) Exchange +Input [2]: [c_customer_sk#14, ca_zip#18] +Arguments: hashpartitioning(c_customer_sk#14, 5), ENSURE_REQUIREMENTS, [id=#20] + +(32) Sort [codegen id : 10] +Input [2]: [c_customer_sk#14, ca_zip#18] +Arguments: [c_customer_sk#14 ASC NULLS FIRST], false, 0 -(38) Exchange -Input [2]: [c_customer_sk#21, ca_zip#25] -Arguments: hashpartitioning(c_customer_sk#21, 5), true, [id=#27] +(33) SortMergeJoin [codegen id : 12] +Left keys [1]: [ss_customer_sk#7] +Right keys [1]: [c_customer_sk#14] +Join condition: NOT (substr(ca_zip#18, 1, 5) = substr(s_zip#11, 1, 5)) -(39) Sort [codegen id : 11] -Input [2]: [c_customer_sk#21, ca_zip#25] -Arguments: [c_customer_sk#21 ASC NULLS FIRST], false, 0 +(34) Project [codegen id : 12] +Output [2]: [ss_item_sk#6, ss_ext_sales_price#9] +Input [6]: [ss_item_sk#6, ss_customer_sk#7, ss_ext_sales_price#9, s_zip#11, c_customer_sk#14, ca_zip#18] -(40) SortMergeJoin [codegen id : 12] -Left keys [1]: [ss_customer_sk#3] -Right keys [1]: [c_customer_sk#21] -Join condition: NOT (substr(ca_zip#25, 1, 5) = substr(s_zip#18, 1, 5)) +(35) Scan parquet default.item +Output [6]: [i_item_sk#21, i_brand_id#22, i_brand#23, i_manufact_id#24, i_manufact#25, i_manager_id#26] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_manager_id), EqualTo(i_manager_id,8), IsNotNull(i_item_sk)] +ReadSchema: struct + +(36) ColumnarToRow [codegen id : 11] +Input [6]: [i_item_sk#21, i_brand_id#22, i_brand#23, i_manufact_id#24, i_manufact#25, i_manager_id#26] + +(37) Filter [codegen id : 11] +Input [6]: [i_item_sk#21, i_brand_id#22, i_brand#23, i_manufact_id#24, i_manufact#25, i_manager_id#26] +Condition : ((isnotnull(i_manager_id#26) AND (i_manager_id#26 = 8)) AND isnotnull(i_item_sk#21)) + +(38) Project [codegen id : 11] +Output [5]: [i_item_sk#21, i_brand_id#22, i_brand#23, i_manufact_id#24, i_manufact#25] +Input [6]: [i_item_sk#21, i_brand_id#22, i_brand#23, i_manufact_id#24, i_manufact#25, i_manager_id#26] + +(39) BroadcastExchange +Input [5]: [i_item_sk#21, i_brand_id#22, i_brand#23, i_manufact_id#24, i_manufact#25] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#27] + +(40) BroadcastHashJoin [codegen id : 12] +Left keys [1]: [ss_item_sk#6] +Right keys [1]: [i_item_sk#21] +Join condition: None (41) Project [codegen id : 12] -Output [5]: [ss_ext_sales_price#5, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10] -Input [9]: [ss_customer_sk#3, ss_ext_sales_price#5, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10, s_zip#18, c_customer_sk#21, ca_zip#25] +Output [5]: [ss_ext_sales_price#9, i_brand_id#22, i_brand#23, i_manufact_id#24, i_manufact#25] +Input [7]: [ss_item_sk#6, ss_ext_sales_price#9, i_item_sk#21, i_brand_id#22, i_brand#23, i_manufact_id#24, i_manufact#25] (42) HashAggregate [codegen id : 12] -Input [5]: [ss_ext_sales_price#5, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10] -Keys [4]: [i_brand#8, i_brand_id#7, i_manufact_id#9, i_manufact#10] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] +Input [5]: [ss_ext_sales_price#9, i_brand_id#22, i_brand#23, i_manufact_id#24, i_manufact#25] +Keys [4]: [i_brand#23, i_brand_id#22, i_manufact_id#24, i_manufact#25] +Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#9))] Aggregate Attributes [1]: [sum#28] -Results [5]: [i_brand#8, i_brand_id#7, i_manufact_id#9, i_manufact#10, sum#29] +Results [5]: [i_brand#23, i_brand_id#22, i_manufact_id#24, i_manufact#25, sum#29] (43) Exchange -Input [5]: [i_brand#8, i_brand_id#7, i_manufact_id#9, i_manufact#10, sum#29] -Arguments: hashpartitioning(i_brand#8, i_brand_id#7, i_manufact_id#9, i_manufact#10, 5), true, [id=#30] +Input [5]: [i_brand#23, i_brand_id#22, i_manufact_id#24, i_manufact#25, sum#29] +Arguments: hashpartitioning(i_brand#23, i_brand_id#22, i_manufact_id#24, i_manufact#25, 5), ENSURE_REQUIREMENTS, [id=#30] (44) HashAggregate [codegen id : 13] -Input [5]: [i_brand#8, i_brand_id#7, i_manufact_id#9, i_manufact#10, sum#29] -Keys [4]: [i_brand#8, i_brand_id#7, i_manufact_id#9, i_manufact#10] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#5))#31] -Results [5]: [i_brand_id#7 AS brand_id#32, i_brand#8 AS brand#33, i_manufact_id#9, i_manufact#10, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#31,17,2) AS ext_price#34] +Input [5]: [i_brand#23, i_brand_id#22, i_manufact_id#24, i_manufact#25, sum#29] +Keys [4]: [i_brand#23, i_brand_id#22, i_manufact_id#24, i_manufact#25] +Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#9))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#9))#31] +Results [5]: [i_brand_id#22 AS brand_id#32, i_brand#23 AS brand#33, i_manufact_id#24, i_manufact#25, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#9))#31,17,2) AS ext_price#34] (45) TakeOrderedAndProject -Input [5]: [brand_id#32, brand#33, i_manufact_id#9, i_manufact#10, ext_price#34] -Arguments: 100, [ext_price#34 DESC NULLS LAST, brand#33 ASC NULLS FIRST, brand_id#32 ASC NULLS FIRST, i_manufact_id#9 ASC NULLS FIRST, i_manufact#10 ASC NULLS FIRST], [brand_id#32, brand#33, i_manufact_id#9, i_manufact#10, ext_price#34] +Input [5]: [brand_id#32, brand#33, i_manufact_id#24, i_manufact#25, ext_price#34] +Arguments: 100, [ext_price#34 DESC NULLS LAST, brand#33 ASC NULLS FIRST, brand_id#32 ASC NULLS FIRST, i_manufact_id#24 ASC NULLS FIRST, i_manufact#25 ASC NULLS FIRST], [brand_id#32, brand#33, i_manufact_id#24, i_manufact#25, ext_price#34] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19.sf100/simplified.txt index 05fa3f82e27df..b6441c5fe72c1 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19.sf100/simplified.txt @@ -6,71 +6,71 @@ TakeOrderedAndProject [ext_price,brand,brand_id,i_manufact_id,i_manufact] WholeStageCodegen (12) HashAggregate [i_brand,i_brand_id,i_manufact_id,i_manufact,ss_ext_sales_price] [sum,sum] Project [ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact] - SortMergeJoin [ss_customer_sk,c_customer_sk,ca_zip,s_zip] - InputAdapter - WholeStageCodegen (5) - Sort [ss_customer_sk] - InputAdapter - Exchange [ss_customer_sk] #2 - WholeStageCodegen (4) - Project [ss_customer_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,s_zip] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_customer_sk,ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_sold_date_sk,ss_item_sk,ss_customer_sk,ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_ext_sales_price] + SortMergeJoin [ss_customer_sk,c_customer_sk,ca_zip,s_zip] + InputAdapter + WholeStageCodegen (4) + Sort [ss_customer_sk] + InputAdapter + Exchange [ss_customer_sk] #2 + WholeStageCodegen (3) + Project [ss_item_sk,ss_customer_sk,ss_ext_sales_price,s_zip] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price] + BroadcastHashJoin [d_date_sk,ss_sold_date_sk] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) - Project [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] - Filter [i_manager_id,i_item_sk] + Project [d_date_sk] + Filter [d_moy,d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact,i_manager_id] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] + Filter [ss_sold_date_sk,ss_item_sk,ss_customer_sk,ss_store_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price] InputAdapter BroadcastExchange #4 WholeStageCodegen (2) - Project [d_date_sk] - Filter [d_moy,d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - Filter [s_zip,s_store_sk] - ColumnarToRow + Filter [s_zip,s_store_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store [s_store_sk,s_zip] + InputAdapter + WholeStageCodegen (10) + Sort [c_customer_sk] + InputAdapter + Exchange [c_customer_sk] #5 + WholeStageCodegen (9) + Project [c_customer_sk,ca_zip] + SortMergeJoin [c_current_addr_sk,ca_address_sk] + InputAdapter + WholeStageCodegen (6) + Sort [c_current_addr_sk] + InputAdapter + Exchange [c_current_addr_sk] #6 + WholeStageCodegen (5) + Filter [c_customer_sk,c_current_addr_sk] + ColumnarToRow + InputAdapter + Scan parquet default.customer [c_customer_sk,c_current_addr_sk] + InputAdapter + WholeStageCodegen (8) + Sort [ca_address_sk] InputAdapter - Scan parquet default.store [s_store_sk,s_zip] + Exchange [ca_address_sk] #7 + WholeStageCodegen (7) + Filter [ca_address_sk,ca_zip] + ColumnarToRow + InputAdapter + Scan parquet default.customer_address [ca_address_sk,ca_zip] InputAdapter - WholeStageCodegen (11) - Sort [c_customer_sk] - InputAdapter - Exchange [c_customer_sk] #6 - WholeStageCodegen (10) - Project [c_customer_sk,ca_zip] - SortMergeJoin [c_current_addr_sk,ca_address_sk] - InputAdapter - WholeStageCodegen (7) - Sort [c_current_addr_sk] - InputAdapter - Exchange [c_current_addr_sk] #7 - WholeStageCodegen (6) - Filter [c_customer_sk,c_current_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_sk,c_current_addr_sk] - InputAdapter - WholeStageCodegen (9) - Sort [ca_address_sk] - InputAdapter - Exchange [ca_address_sk] #8 - WholeStageCodegen (8) - Filter [ca_address_sk,ca_zip] - ColumnarToRow - InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_zip] + BroadcastExchange #8 + WholeStageCodegen (11) + Project [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] + Filter [i_manager_id,i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact,i_manager_id] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.sf100/explain.txt index ffcf6bd4f6d47..093c4eed6cf11 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.sf100/explain.txt @@ -92,7 +92,7 @@ Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, (10) Exchange Input [10]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] -Arguments: hashpartitioning(ss_customer_sk#2, 5), true, [id=#13] +Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [id=#13] (11) Sort [codegen id : 3] Input [10]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] @@ -114,7 +114,7 @@ Condition : (isnotnull(c_customer_sk#14) AND isnotnull(c_birth_country#17)) (15) Exchange Input [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] -Arguments: hashpartitioning(c_customer_sk#14, 5), true, [id=#18] +Arguments: hashpartitioning(c_customer_sk#14, 5), ENSURE_REQUIREMENTS, [id=#18] (16) Sort [codegen id : 5] Input [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] @@ -189,7 +189,7 @@ Input [17]: [ss_item_sk#1, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, i_c (32) Exchange Input [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, s_store_name#20, s_state#22, ca_state#25] -Arguments: hashpartitioning(cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint), 5), true, [id=#29] +Arguments: hashpartitioning(cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint), 5), ENSURE_REQUIREMENTS, [id=#29] (33) Sort [codegen id : 9] Input [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, s_store_name#20, s_state#22, ca_state#25] @@ -211,7 +211,7 @@ Condition : (isnotnull(sr_ticket_number#31) AND isnotnull(sr_item_sk#30)) (37) Exchange Input [2]: [sr_item_sk#30, sr_ticket_number#31] -Arguments: hashpartitioning(sr_ticket_number#31, sr_item_sk#30, 5), true, [id=#32] +Arguments: hashpartitioning(sr_ticket_number#31, sr_item_sk#30, 5), ENSURE_REQUIREMENTS, [id=#32] (38) Sort [codegen id : 11] Input [2]: [sr_item_sk#30, sr_ticket_number#31] @@ -235,7 +235,7 @@ Results [11]: [c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_ (42) Exchange Input [11]: [c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_state#22, i_color#9, i_current_price#7, i_manager_id#11, i_units#10, i_size#8, sum#34] -Arguments: hashpartitioning(c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_state#22, i_color#9, i_current_price#7, i_manager_id#11, i_units#10, i_size#8, 5), true, [id=#35] +Arguments: hashpartitioning(c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_state#22, i_color#9, i_current_price#7, i_manager_id#11, i_units#10, i_size#8, 5), ENSURE_REQUIREMENTS, [id=#35] (43) HashAggregate [codegen id : 13] Input [11]: [c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_state#22, i_color#9, i_current_price#7, i_manager_id#11, i_units#10, i_size#8, sum#34] @@ -253,7 +253,7 @@ Results [5]: [c_last_name#16, c_first_name#15, s_store_name#20, sum#40, isEmpty# (45) Exchange Input [5]: [c_last_name#16, c_first_name#15, s_store_name#20, sum#40, isEmpty#41] -Arguments: hashpartitioning(c_last_name#16, c_first_name#15, s_store_name#20, 5), true, [id=#42] +Arguments: hashpartitioning(c_last_name#16, c_first_name#15, s_store_name#20, 5), ENSURE_REQUIREMENTS, [id=#42] (46) HashAggregate [codegen id : 14] Input [5]: [c_last_name#16, c_first_name#15, s_store_name#20, sum#40, isEmpty#41] @@ -309,12 +309,12 @@ Subquery:1 Hosting operator id = 47 Hosting Expression = Subquery scalar-subquer : : : +- Exchange (64) : : : +- * Filter (63) : : : +- * ColumnarToRow (62) - : : : +- Scan parquet default.item (61) + : : : +- Scan parquet default.customer (61) : : +- * Sort (74) : : +- Exchange (73) : : +- * Filter (72) : : +- * ColumnarToRow (71) - : : +- Scan parquet default.customer (70) + : : +- Scan parquet default.item (70) : +- * Sort (83) : +- Exchange (82) : +- * Filter (81) @@ -374,88 +374,88 @@ Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, s (59) Exchange Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23] -Arguments: hashpartitioning(ss_item_sk#1, 5), true, [id=#49] +Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [id=#49] (60) Sort [codegen id : 3] Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23] -Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0 +Arguments: [ss_customer_sk#2 ASC NULLS FIRST], false, 0 -(61) Scan parquet default.item -Output [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] +(61) Scan parquet default.customer +Output [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/customer] +PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_birth_country)] +ReadSchema: struct (62) ColumnarToRow [codegen id : 4] -Input [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] +Input [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] (63) Filter [codegen id : 4] -Input [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] -Condition : isnotnull(i_item_sk#6) +Input [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] +Condition : (isnotnull(c_customer_sk#14) AND isnotnull(c_birth_country#17)) (64) Exchange -Input [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] -Arguments: hashpartitioning(i_item_sk#6, 5), true, [id=#50] +Input [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] +Arguments: hashpartitioning(c_customer_sk#14, 5), ENSURE_REQUIREMENTS, [id=#50] (65) Sort [codegen id : 5] -Input [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] -Arguments: [i_item_sk#6 ASC NULLS FIRST], false, 0 +Input [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] +Arguments: [c_customer_sk#14 ASC NULLS FIRST], false, 0 (66) SortMergeJoin [codegen id : 6] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#6] +Left keys [1]: [ss_customer_sk#2] +Right keys [1]: [c_customer_sk#14] Join condition: None (67) Project [codegen id : 6] -Output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] -Input [13]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] +Output [9]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, c_first_name#15, c_last_name#16, c_birth_country#17] +Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] (68) Exchange -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] -Arguments: hashpartitioning(ss_customer_sk#2, 5), true, [id=#51] +Input [9]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, c_first_name#15, c_last_name#16, c_birth_country#17] +Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#51] (69) Sort [codegen id : 7] -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] -Arguments: [ss_customer_sk#2 ASC NULLS FIRST], false, 0 +Input [9]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, c_first_name#15, c_last_name#16, c_birth_country#17] +Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0 -(70) Scan parquet default.customer -Output [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] +(70) Scan parquet default.item +Output [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_birth_country)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_item_sk)] +ReadSchema: struct (71) ColumnarToRow [codegen id : 8] -Input [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] +Input [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] (72) Filter [codegen id : 8] -Input [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] -Condition : (isnotnull(c_customer_sk#14) AND isnotnull(c_birth_country#17)) +Input [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] +Condition : isnotnull(i_item_sk#6) (73) Exchange -Input [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] -Arguments: hashpartitioning(c_customer_sk#14, 5), true, [id=#52] +Input [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] +Arguments: hashpartitioning(i_item_sk#6, 5), ENSURE_REQUIREMENTS, [id=#52] (74) Sort [codegen id : 9] -Input [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] -Arguments: [c_customer_sk#14 ASC NULLS FIRST], false, 0 +Input [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] +Arguments: [i_item_sk#6 ASC NULLS FIRST], false, 0 (75) SortMergeJoin [codegen id : 10] -Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#14] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#6] Join condition: None (76) Project [codegen id : 10] -Output [14]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, c_birth_country#17] -Input [16]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] +Output [14]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, c_first_name#15, c_last_name#16, c_birth_country#17, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] +Input [15]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, c_first_name#15, c_last_name#16, c_birth_country#17, i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] (77) Exchange -Input [14]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, c_birth_country#17] -Arguments: hashpartitioning(c_birth_country#17, s_zip#23, 5), true, [id=#53] +Input [14]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, c_first_name#15, c_last_name#16, c_birth_country#17, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] +Arguments: hashpartitioning(c_birth_country#17, s_zip#23, 5), ENSURE_REQUIREMENTS, [id=#53] (78) Sort [codegen id : 11] -Input [14]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, c_birth_country#17] +Input [14]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, c_first_name#15, c_last_name#16, c_birth_country#17, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] Arguments: [c_birth_country#17 ASC NULLS FIRST, s_zip#23 ASC NULLS FIRST], false, 0 (79) Scan parquet default.customer_address @@ -474,7 +474,7 @@ Condition : (isnotnull(ca_country#27) AND isnotnull(ca_zip#26)) (82) Exchange Input [3]: [ca_state#25, ca_zip#26, ca_country#27] -Arguments: hashpartitioning(upper(ca_country#27), ca_zip#26, 5), true, [id=#54] +Arguments: hashpartitioning(upper(ca_country#27), ca_zip#26, 5), ENSURE_REQUIREMENTS, [id=#54] (83) Sort [codegen id : 13] Input [3]: [ca_state#25, ca_zip#26, ca_country#27] @@ -486,15 +486,15 @@ Right keys [2]: [upper(ca_country#27), ca_zip#26] Join condition: None (85) Project [codegen id : 14] -Output [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, ca_state#25] -Input [17]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, c_birth_country#17, ca_state#25, ca_zip#26, ca_country#27] +Output [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, c_first_name#15, c_last_name#16, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, ca_state#25] +Input [17]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, c_first_name#15, c_last_name#16, c_birth_country#17, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, ca_state#25, ca_zip#26, ca_country#27] (86) Exchange -Input [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, ca_state#25] -Arguments: hashpartitioning(cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint), 5), true, [id=#55] +Input [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, c_first_name#15, c_last_name#16, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, ca_state#25] +Arguments: hashpartitioning(cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint), 5), ENSURE_REQUIREMENTS, [id=#55] (87) Sort [codegen id : 15] -Input [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, ca_state#25] +Input [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, c_first_name#15, c_last_name#16, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, ca_state#25] Arguments: [cast(ss_ticket_number#4 as bigint) ASC NULLS FIRST, cast(ss_item_sk#1 as bigint) ASC NULLS FIRST], false, 0 (88) Scan parquet default.store_returns @@ -513,7 +513,7 @@ Condition : (isnotnull(sr_ticket_number#31) AND isnotnull(sr_item_sk#30)) (91) Exchange Input [2]: [sr_item_sk#30, sr_ticket_number#31] -Arguments: hashpartitioning(sr_ticket_number#31, sr_item_sk#30, 5), true, [id=#56] +Arguments: hashpartitioning(sr_ticket_number#31, sr_item_sk#30, 5), ENSURE_REQUIREMENTS, [id=#56] (92) Sort [codegen id : 17] Input [2]: [sr_item_sk#30, sr_ticket_number#31] @@ -526,7 +526,7 @@ Join condition: None (94) Project [codegen id : 18] Output [11]: [ss_net_paid#5, s_store_name#20, s_state#22, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, ca_state#25] -Input [15]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, ca_state#25, sr_item_sk#30, sr_ticket_number#31] +Input [15]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, c_first_name#15, c_last_name#16, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, ca_state#25, sr_item_sk#30, sr_ticket_number#31] (95) HashAggregate [codegen id : 18] Input [11]: [ss_net_paid#5, s_store_name#20, s_state#22, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, ca_state#25] @@ -537,7 +537,7 @@ Results [11]: [c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_ (96) Exchange Input [11]: [c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_state#22, i_color#9, i_current_price#7, i_manager_id#11, i_units#10, i_size#8, sum#58] -Arguments: hashpartitioning(c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_state#22, i_color#9, i_current_price#7, i_manager_id#11, i_units#10, i_size#8, 5), true, [id=#59] +Arguments: hashpartitioning(c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_state#22, i_color#9, i_current_price#7, i_manager_id#11, i_units#10, i_size#8, 5), ENSURE_REQUIREMENTS, [id=#59] (97) HashAggregate [codegen id : 19] Input [11]: [c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_state#22, i_color#9, i_current_price#7, i_manager_id#11, i_units#10, i_size#8, sum#58] @@ -555,7 +555,7 @@ Results [2]: [sum#63, count#64] (99) Exchange Input [2]: [sum#63, count#64] -Arguments: SinglePartition, true, [id=#65] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#65] (100) HashAggregate [codegen id : 20] Input [2]: [sum#63, count#64] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.sf100/simplified.txt index 10f874f8f5543..7de562c5d59a1 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.sf100/simplified.txt @@ -21,7 +21,7 @@ WholeStageCodegen (14) InputAdapter Exchange [ss_ticket_number,ss_item_sk] #12 WholeStageCodegen (14) - Project [ss_item_sk,ss_ticket_number,ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] + Project [ss_item_sk,ss_ticket_number,ss_net_paid,s_store_name,s_state,c_first_name,c_last_name,i_current_price,i_size,i_color,i_units,i_manager_id,ca_state] SortMergeJoin [c_birth_country,s_zip,ca_country,ca_zip] InputAdapter WholeStageCodegen (11) @@ -29,21 +29,21 @@ WholeStageCodegen (14) InputAdapter Exchange [c_birth_country,s_zip] #13 WholeStageCodegen (10) - Project [ss_item_sk,ss_ticket_number,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] - SortMergeJoin [ss_customer_sk,c_customer_sk] + Project [ss_item_sk,ss_ticket_number,ss_net_paid,s_store_name,s_state,s_zip,c_first_name,c_last_name,c_birth_country,i_current_price,i_size,i_color,i_units,i_manager_id] + SortMergeJoin [ss_item_sk,i_item_sk] InputAdapter WholeStageCodegen (7) - Sort [ss_customer_sk] + Sort [ss_item_sk] InputAdapter - Exchange [ss_customer_sk] #14 + Exchange [ss_item_sk] #14 WholeStageCodegen (6) - Project [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] - SortMergeJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_ticket_number,ss_net_paid,s_store_name,s_state,s_zip,c_first_name,c_last_name,c_birth_country] + SortMergeJoin [ss_customer_sk,c_customer_sk] InputAdapter WholeStageCodegen (3) - Sort [ss_item_sk] + Sort [ss_customer_sk] InputAdapter - Exchange [ss_item_sk] #15 + Exchange [ss_customer_sk] #15 WholeStageCodegen (2) Project [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_net_paid,s_store_name,s_state,s_zip] BroadcastHashJoin [ss_store_sk,s_store_sk] @@ -61,24 +61,24 @@ WholeStageCodegen (14) Scan parquet default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] InputAdapter WholeStageCodegen (5) - Sort [i_item_sk] + Sort [c_customer_sk] InputAdapter - Exchange [i_item_sk] #17 + Exchange [c_customer_sk] #17 WholeStageCodegen (4) - Filter [i_item_sk] + Filter [c_customer_sk,c_birth_country] ColumnarToRow InputAdapter - Scan parquet default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] InputAdapter WholeStageCodegen (9) - Sort [c_customer_sk] + Sort [i_item_sk] InputAdapter - Exchange [c_customer_sk] #18 + Exchange [i_item_sk] #18 WholeStageCodegen (8) - Filter [c_customer_sk,c_birth_country] + Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] + Scan parquet default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] InputAdapter WholeStageCodegen (13) Sort [ca_country,ca_zip] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.sf100/explain.txt index 73f36e3a9ca23..273950bed3546 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.sf100/explain.txt @@ -92,7 +92,7 @@ Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, (10) Exchange Input [10]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] -Arguments: hashpartitioning(ss_customer_sk#2, 5), true, [id=#13] +Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [id=#13] (11) Sort [codegen id : 3] Input [10]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] @@ -114,7 +114,7 @@ Condition : (isnotnull(c_customer_sk#14) AND isnotnull(c_birth_country#17)) (15) Exchange Input [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] -Arguments: hashpartitioning(c_customer_sk#14, 5), true, [id=#18] +Arguments: hashpartitioning(c_customer_sk#14, 5), ENSURE_REQUIREMENTS, [id=#18] (16) Sort [codegen id : 5] Input [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] @@ -189,7 +189,7 @@ Input [17]: [ss_item_sk#1, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, i_c (32) Exchange Input [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, s_store_name#20, s_state#22, ca_state#25] -Arguments: hashpartitioning(cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint), 5), true, [id=#29] +Arguments: hashpartitioning(cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint), 5), ENSURE_REQUIREMENTS, [id=#29] (33) Sort [codegen id : 9] Input [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, s_store_name#20, s_state#22, ca_state#25] @@ -211,7 +211,7 @@ Condition : (isnotnull(sr_ticket_number#31) AND isnotnull(sr_item_sk#30)) (37) Exchange Input [2]: [sr_item_sk#30, sr_ticket_number#31] -Arguments: hashpartitioning(sr_ticket_number#31, sr_item_sk#30, 5), true, [id=#32] +Arguments: hashpartitioning(sr_ticket_number#31, sr_item_sk#30, 5), ENSURE_REQUIREMENTS, [id=#32] (38) Sort [codegen id : 11] Input [2]: [sr_item_sk#30, sr_ticket_number#31] @@ -235,7 +235,7 @@ Results [11]: [c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_ (42) Exchange Input [11]: [c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_state#22, i_color#9, i_current_price#7, i_manager_id#11, i_units#10, i_size#8, sum#34] -Arguments: hashpartitioning(c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_state#22, i_color#9, i_current_price#7, i_manager_id#11, i_units#10, i_size#8, 5), true, [id=#35] +Arguments: hashpartitioning(c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_state#22, i_color#9, i_current_price#7, i_manager_id#11, i_units#10, i_size#8, 5), ENSURE_REQUIREMENTS, [id=#35] (43) HashAggregate [codegen id : 13] Input [11]: [c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_state#22, i_color#9, i_current_price#7, i_manager_id#11, i_units#10, i_size#8, sum#34] @@ -253,7 +253,7 @@ Results [5]: [c_last_name#16, c_first_name#15, s_store_name#20, sum#40, isEmpty# (45) Exchange Input [5]: [c_last_name#16, c_first_name#15, s_store_name#20, sum#40, isEmpty#41] -Arguments: hashpartitioning(c_last_name#16, c_first_name#15, s_store_name#20, 5), true, [id=#42] +Arguments: hashpartitioning(c_last_name#16, c_first_name#15, s_store_name#20, 5), ENSURE_REQUIREMENTS, [id=#42] (46) HashAggregate [codegen id : 14] Input [5]: [c_last_name#16, c_first_name#15, s_store_name#20, sum#40, isEmpty#41] @@ -309,12 +309,12 @@ Subquery:1 Hosting operator id = 47 Hosting Expression = Subquery scalar-subquer : : : +- Exchange (64) : : : +- * Filter (63) : : : +- * ColumnarToRow (62) - : : : +- Scan parquet default.item (61) + : : : +- Scan parquet default.customer (61) : : +- * Sort (74) : : +- Exchange (73) : : +- * Filter (72) : : +- * ColumnarToRow (71) - : : +- Scan parquet default.customer (70) + : : +- Scan parquet default.item (70) : +- * Sort (83) : +- Exchange (82) : +- * Filter (81) @@ -374,88 +374,88 @@ Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, s (59) Exchange Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23] -Arguments: hashpartitioning(ss_item_sk#1, 5), true, [id=#49] +Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [id=#49] (60) Sort [codegen id : 3] Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23] -Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0 +Arguments: [ss_customer_sk#2 ASC NULLS FIRST], false, 0 -(61) Scan parquet default.item -Output [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] +(61) Scan parquet default.customer +Output [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/customer] +PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_birth_country)] +ReadSchema: struct (62) ColumnarToRow [codegen id : 4] -Input [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] +Input [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] (63) Filter [codegen id : 4] -Input [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] -Condition : isnotnull(i_item_sk#6) +Input [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] +Condition : (isnotnull(c_customer_sk#14) AND isnotnull(c_birth_country#17)) (64) Exchange -Input [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] -Arguments: hashpartitioning(i_item_sk#6, 5), true, [id=#50] +Input [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] +Arguments: hashpartitioning(c_customer_sk#14, 5), ENSURE_REQUIREMENTS, [id=#50] (65) Sort [codegen id : 5] -Input [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] -Arguments: [i_item_sk#6 ASC NULLS FIRST], false, 0 +Input [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] +Arguments: [c_customer_sk#14 ASC NULLS FIRST], false, 0 (66) SortMergeJoin [codegen id : 6] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#6] +Left keys [1]: [ss_customer_sk#2] +Right keys [1]: [c_customer_sk#14] Join condition: None (67) Project [codegen id : 6] -Output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] -Input [13]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] +Output [9]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, c_first_name#15, c_last_name#16, c_birth_country#17] +Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] (68) Exchange -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] -Arguments: hashpartitioning(ss_customer_sk#2, 5), true, [id=#51] +Input [9]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, c_first_name#15, c_last_name#16, c_birth_country#17] +Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#51] (69) Sort [codegen id : 7] -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] -Arguments: [ss_customer_sk#2 ASC NULLS FIRST], false, 0 +Input [9]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, c_first_name#15, c_last_name#16, c_birth_country#17] +Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0 -(70) Scan parquet default.customer -Output [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] +(70) Scan parquet default.item +Output [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_birth_country)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_item_sk)] +ReadSchema: struct (71) ColumnarToRow [codegen id : 8] -Input [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] +Input [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] (72) Filter [codegen id : 8] -Input [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] -Condition : (isnotnull(c_customer_sk#14) AND isnotnull(c_birth_country#17)) +Input [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] +Condition : isnotnull(i_item_sk#6) (73) Exchange -Input [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] -Arguments: hashpartitioning(c_customer_sk#14, 5), true, [id=#52] +Input [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] +Arguments: hashpartitioning(i_item_sk#6, 5), ENSURE_REQUIREMENTS, [id=#52] (74) Sort [codegen id : 9] -Input [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] -Arguments: [c_customer_sk#14 ASC NULLS FIRST], false, 0 +Input [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] +Arguments: [i_item_sk#6 ASC NULLS FIRST], false, 0 (75) SortMergeJoin [codegen id : 10] -Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#14] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#6] Join condition: None (76) Project [codegen id : 10] -Output [14]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, c_birth_country#17] -Input [16]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] +Output [14]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, c_first_name#15, c_last_name#16, c_birth_country#17, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] +Input [15]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, c_first_name#15, c_last_name#16, c_birth_country#17, i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] (77) Exchange -Input [14]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, c_birth_country#17] -Arguments: hashpartitioning(c_birth_country#17, s_zip#23, 5), true, [id=#53] +Input [14]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, c_first_name#15, c_last_name#16, c_birth_country#17, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] +Arguments: hashpartitioning(c_birth_country#17, s_zip#23, 5), ENSURE_REQUIREMENTS, [id=#53] (78) Sort [codegen id : 11] -Input [14]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, c_birth_country#17] +Input [14]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, c_first_name#15, c_last_name#16, c_birth_country#17, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] Arguments: [c_birth_country#17 ASC NULLS FIRST, s_zip#23 ASC NULLS FIRST], false, 0 (79) Scan parquet default.customer_address @@ -474,7 +474,7 @@ Condition : (isnotnull(ca_country#27) AND isnotnull(ca_zip#26)) (82) Exchange Input [3]: [ca_state#25, ca_zip#26, ca_country#27] -Arguments: hashpartitioning(upper(ca_country#27), ca_zip#26, 5), true, [id=#54] +Arguments: hashpartitioning(upper(ca_country#27), ca_zip#26, 5), ENSURE_REQUIREMENTS, [id=#54] (83) Sort [codegen id : 13] Input [3]: [ca_state#25, ca_zip#26, ca_country#27] @@ -486,15 +486,15 @@ Right keys [2]: [upper(ca_country#27), ca_zip#26] Join condition: None (85) Project [codegen id : 14] -Output [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, ca_state#25] -Input [17]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, c_birth_country#17, ca_state#25, ca_zip#26, ca_country#27] +Output [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, c_first_name#15, c_last_name#16, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, ca_state#25] +Input [17]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, c_first_name#15, c_last_name#16, c_birth_country#17, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, ca_state#25, ca_zip#26, ca_country#27] (86) Exchange -Input [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, ca_state#25] -Arguments: hashpartitioning(cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint), 5), true, [id=#55] +Input [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, c_first_name#15, c_last_name#16, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, ca_state#25] +Arguments: hashpartitioning(cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint), 5), ENSURE_REQUIREMENTS, [id=#55] (87) Sort [codegen id : 15] -Input [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, ca_state#25] +Input [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, c_first_name#15, c_last_name#16, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, ca_state#25] Arguments: [cast(ss_ticket_number#4 as bigint) ASC NULLS FIRST, cast(ss_item_sk#1 as bigint) ASC NULLS FIRST], false, 0 (88) Scan parquet default.store_returns @@ -513,7 +513,7 @@ Condition : (isnotnull(sr_ticket_number#31) AND isnotnull(sr_item_sk#30)) (91) Exchange Input [2]: [sr_item_sk#30, sr_ticket_number#31] -Arguments: hashpartitioning(sr_ticket_number#31, sr_item_sk#30, 5), true, [id=#56] +Arguments: hashpartitioning(sr_ticket_number#31, sr_item_sk#30, 5), ENSURE_REQUIREMENTS, [id=#56] (92) Sort [codegen id : 17] Input [2]: [sr_item_sk#30, sr_ticket_number#31] @@ -526,7 +526,7 @@ Join condition: None (94) Project [codegen id : 18] Output [11]: [ss_net_paid#5, s_store_name#20, s_state#22, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, ca_state#25] -Input [15]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, ca_state#25, sr_item_sk#30, sr_ticket_number#31] +Input [15]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, c_first_name#15, c_last_name#16, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, ca_state#25, sr_item_sk#30, sr_ticket_number#31] (95) HashAggregate [codegen id : 18] Input [11]: [ss_net_paid#5, s_store_name#20, s_state#22, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, ca_state#25] @@ -537,7 +537,7 @@ Results [11]: [c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_ (96) Exchange Input [11]: [c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_state#22, i_color#9, i_current_price#7, i_manager_id#11, i_units#10, i_size#8, sum#58] -Arguments: hashpartitioning(c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_state#22, i_color#9, i_current_price#7, i_manager_id#11, i_units#10, i_size#8, 5), true, [id=#59] +Arguments: hashpartitioning(c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_state#22, i_color#9, i_current_price#7, i_manager_id#11, i_units#10, i_size#8, 5), ENSURE_REQUIREMENTS, [id=#59] (97) HashAggregate [codegen id : 19] Input [11]: [c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_state#22, i_color#9, i_current_price#7, i_manager_id#11, i_units#10, i_size#8, sum#58] @@ -555,7 +555,7 @@ Results [2]: [sum#63, count#64] (99) Exchange Input [2]: [sum#63, count#64] -Arguments: SinglePartition, true, [id=#65] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#65] (100) HashAggregate [codegen id : 20] Input [2]: [sum#63, count#64] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.sf100/simplified.txt index 10f874f8f5543..7de562c5d59a1 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.sf100/simplified.txt @@ -21,7 +21,7 @@ WholeStageCodegen (14) InputAdapter Exchange [ss_ticket_number,ss_item_sk] #12 WholeStageCodegen (14) - Project [ss_item_sk,ss_ticket_number,ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] + Project [ss_item_sk,ss_ticket_number,ss_net_paid,s_store_name,s_state,c_first_name,c_last_name,i_current_price,i_size,i_color,i_units,i_manager_id,ca_state] SortMergeJoin [c_birth_country,s_zip,ca_country,ca_zip] InputAdapter WholeStageCodegen (11) @@ -29,21 +29,21 @@ WholeStageCodegen (14) InputAdapter Exchange [c_birth_country,s_zip] #13 WholeStageCodegen (10) - Project [ss_item_sk,ss_ticket_number,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] - SortMergeJoin [ss_customer_sk,c_customer_sk] + Project [ss_item_sk,ss_ticket_number,ss_net_paid,s_store_name,s_state,s_zip,c_first_name,c_last_name,c_birth_country,i_current_price,i_size,i_color,i_units,i_manager_id] + SortMergeJoin [ss_item_sk,i_item_sk] InputAdapter WholeStageCodegen (7) - Sort [ss_customer_sk] + Sort [ss_item_sk] InputAdapter - Exchange [ss_customer_sk] #14 + Exchange [ss_item_sk] #14 WholeStageCodegen (6) - Project [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] - SortMergeJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_ticket_number,ss_net_paid,s_store_name,s_state,s_zip,c_first_name,c_last_name,c_birth_country] + SortMergeJoin [ss_customer_sk,c_customer_sk] InputAdapter WholeStageCodegen (3) - Sort [ss_item_sk] + Sort [ss_customer_sk] InputAdapter - Exchange [ss_item_sk] #15 + Exchange [ss_customer_sk] #15 WholeStageCodegen (2) Project [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_net_paid,s_store_name,s_state,s_zip] BroadcastHashJoin [ss_store_sk,s_store_sk] @@ -61,24 +61,24 @@ WholeStageCodegen (14) Scan parquet default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] InputAdapter WholeStageCodegen (5) - Sort [i_item_sk] + Sort [c_customer_sk] InputAdapter - Exchange [i_item_sk] #17 + Exchange [c_customer_sk] #17 WholeStageCodegen (4) - Filter [i_item_sk] + Filter [c_customer_sk,c_birth_country] ColumnarToRow InputAdapter - Scan parquet default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] InputAdapter WholeStageCodegen (9) - Sort [c_customer_sk] + Sort [i_item_sk] InputAdapter - Exchange [c_customer_sk] #18 + Exchange [i_item_sk] #18 WholeStageCodegen (8) - Filter [c_customer_sk,c_birth_country] + Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] + Scan parquet default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] InputAdapter WholeStageCodegen (13) Sort [ca_country,ca_zip] 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 c6dc3db869003..3100e574e60e3 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 @@ -5,57 +5,57 @@ TakeOrderedAndProject (57) +- * HashAggregate (54) +- * Project (53) +- * SortMergeJoin Inner (52) - :- * Sort (43) - : +- Exchange (42) - : +- * Project (41) - : +- * SortMergeJoin Inner (40) - : :- * Sort (27) - : : +- Exchange (26) - : : +- * Project (25) - : : +- * SortMergeJoin Inner (24) - : : :- * Sort (18) - : : : +- Exchange (17) - : : : +- * Project (16) - : : : +- * BroadcastHashJoin Inner BuildRight (15) - : : : :- * Project (10) - : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet default.store_sales (1) - : : : : +- BroadcastExchange (8) - : : : : +- * Project (7) - : : : : +- * Filter (6) - : : : : +- * ColumnarToRow (5) - : : : : +- Scan parquet default.date_dim (4) - : : : +- BroadcastExchange (14) - : : : +- * Filter (13) - : : : +- * ColumnarToRow (12) - : : : +- Scan parquet default.store (11) - : : +- * Sort (23) - : : +- Exchange (22) - : : +- * Filter (21) - : : +- * ColumnarToRow (20) - : : +- Scan parquet default.item (19) - : +- * Sort (39) - : +- Exchange (38) - : +- * Project (37) - : +- * BroadcastHashJoin Inner BuildRight (36) - : :- * Filter (30) - : : +- * ColumnarToRow (29) - : : +- Scan parquet default.store_returns (28) - : +- BroadcastExchange (35) - : +- * Project (34) - : +- * Filter (33) - : +- * ColumnarToRow (32) - : +- Scan parquet default.date_dim (31) + :- * Sort (27) + : +- Exchange (26) + : +- * Project (25) + : +- * SortMergeJoin Inner (24) + : :- * Sort (18) + : : +- Exchange (17) + : : +- * Project (16) + : : +- * BroadcastHashJoin Inner BuildRight (15) + : : :- * Project (10) + : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : :- * Filter (3) + : : : : +- * ColumnarToRow (2) + : : : : +- Scan parquet default.store_sales (1) + : : : +- BroadcastExchange (8) + : : : +- * Project (7) + : : : +- * Filter (6) + : : : +- * ColumnarToRow (5) + : : : +- Scan parquet default.date_dim (4) + : : +- BroadcastExchange (14) + : : +- * Filter (13) + : : +- * ColumnarToRow (12) + : : +- Scan parquet default.store (11) + : +- * Sort (23) + : +- Exchange (22) + : +- * Filter (21) + : +- * ColumnarToRow (20) + : +- Scan parquet default.item (19) +- * Sort (51) +- Exchange (50) +- * Project (49) - +- * BroadcastHashJoin Inner BuildRight (48) - :- * Filter (46) - : +- * ColumnarToRow (45) - : +- Scan parquet default.catalog_sales (44) - +- ReusedExchange (47) + +- * SortMergeJoin Inner (48) + :- * Sort (39) + : +- Exchange (38) + : +- * Project (37) + : +- * BroadcastHashJoin Inner BuildRight (36) + : :- * Filter (30) + : : +- * ColumnarToRow (29) + : : +- Scan parquet default.store_returns (28) + : +- BroadcastExchange (35) + : +- * Project (34) + : +- * Filter (33) + : +- * ColumnarToRow (32) + : +- Scan parquet default.date_dim (31) + +- * Sort (47) + +- Exchange (46) + +- * Project (45) + +- * BroadcastHashJoin Inner BuildRight (44) + :- * Filter (42) + : +- * ColumnarToRow (41) + : +- Scan parquet default.catalog_sales (40) + +- ReusedExchange (43) (1) Scan parquet default.store_sales @@ -132,7 +132,7 @@ Input [8]: [ss_item_sk#2, ss_customer_sk#3, ss_store_sk#4, ss_ticket_number#5, s (17) Exchange Input [6]: [ss_item_sk#2, ss_customer_sk#3, ss_ticket_number#5, ss_net_profit#6, s_store_id#12, s_store_name#13] -Arguments: hashpartitioning(ss_item_sk#2, 5), true, [id=#15] +Arguments: hashpartitioning(ss_item_sk#2, 5), ENSURE_REQUIREMENTS, [id=#15] (18) Sort [codegen id : 4] Input [6]: [ss_item_sk#2, ss_customer_sk#3, ss_ticket_number#5, ss_net_profit#6, s_store_id#12, s_store_name#13] @@ -154,7 +154,7 @@ Condition : isnotnull(i_item_sk#16) (22) Exchange Input [3]: [i_item_sk#16, i_item_id#17, i_item_desc#18] -Arguments: hashpartitioning(i_item_sk#16, 5), true, [id=#19] +Arguments: hashpartitioning(i_item_sk#16, 5), ENSURE_REQUIREMENTS, [id=#19] (23) Sort [codegen id : 6] Input [3]: [i_item_sk#16, i_item_id#17, i_item_desc#18] @@ -171,7 +171,7 @@ Input [9]: [ss_item_sk#2, ss_customer_sk#3, ss_ticket_number#5, ss_net_profit#6, (26) Exchange Input [8]: [ss_item_sk#2, ss_customer_sk#3, ss_ticket_number#5, ss_net_profit#6, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18] -Arguments: hashpartitioning(cast(ss_customer_sk#3 as bigint), cast(ss_item_sk#2 as bigint), cast(ss_ticket_number#5 as bigint), 5), true, [id=#20] +Arguments: hashpartitioning(cast(ss_customer_sk#3 as bigint), cast(ss_item_sk#2 as bigint), cast(ss_ticket_number#5 as bigint), 5), ENSURE_REQUIREMENTS, [id=#20] (27) Sort [codegen id : 8] Input [8]: [ss_item_sk#2, ss_customer_sk#3, ss_ticket_number#5, ss_net_profit#6, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18] @@ -224,89 +224,89 @@ Input [6]: [sr_returned_date_sk#21, sr_item_sk#22, sr_customer_sk#23, sr_ticket_ (38) Exchange Input [4]: [sr_item_sk#22, sr_customer_sk#23, sr_ticket_number#24, sr_net_loss#25] -Arguments: hashpartitioning(sr_customer_sk#23, sr_item_sk#22, sr_ticket_number#24, 5), true, [id=#30] +Arguments: hashpartitioning(sr_customer_sk#23, sr_item_sk#22, 5), ENSURE_REQUIREMENTS, [id=#30] (39) Sort [codegen id : 11] Input [4]: [sr_item_sk#22, sr_customer_sk#23, sr_ticket_number#24, sr_net_loss#25] -Arguments: [sr_customer_sk#23 ASC NULLS FIRST, sr_item_sk#22 ASC NULLS FIRST, sr_ticket_number#24 ASC NULLS FIRST], false, 0 - -(40) SortMergeJoin [codegen id : 12] -Left keys [3]: [cast(ss_customer_sk#3 as bigint), cast(ss_item_sk#2 as bigint), cast(ss_ticket_number#5 as bigint)] -Right keys [3]: [sr_customer_sk#23, sr_item_sk#22, sr_ticket_number#24] -Join condition: None - -(41) Project [codegen id : 12] -Output [8]: [ss_net_profit#6, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18, sr_item_sk#22, sr_customer_sk#23, sr_net_loss#25] -Input [12]: [ss_item_sk#2, ss_customer_sk#3, ss_ticket_number#5, ss_net_profit#6, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18, sr_item_sk#22, sr_customer_sk#23, sr_ticket_number#24, sr_net_loss#25] - -(42) Exchange -Input [8]: [ss_net_profit#6, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18, sr_item_sk#22, sr_customer_sk#23, sr_net_loss#25] -Arguments: hashpartitioning(sr_customer_sk#23, sr_item_sk#22, 5), true, [id=#31] - -(43) Sort [codegen id : 13] -Input [8]: [ss_net_profit#6, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18, sr_item_sk#22, sr_customer_sk#23, sr_net_loss#25] Arguments: [sr_customer_sk#23 ASC NULLS FIRST, sr_item_sk#22 ASC NULLS FIRST], false, 0 -(44) Scan parquet default.catalog_sales -Output [4]: [cs_sold_date_sk#32, cs_bill_customer_sk#33, cs_item_sk#34, cs_net_profit#35] +(40) Scan parquet default.catalog_sales +Output [4]: [cs_sold_date_sk#31, cs_bill_customer_sk#32, cs_item_sk#33, cs_net_profit#34] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_sales] PushedFilters: [IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk), IsNotNull(cs_sold_date_sk)] ReadSchema: struct -(45) ColumnarToRow [codegen id : 15] -Input [4]: [cs_sold_date_sk#32, cs_bill_customer_sk#33, cs_item_sk#34, cs_net_profit#35] +(41) ColumnarToRow [codegen id : 13] +Input [4]: [cs_sold_date_sk#31, cs_bill_customer_sk#32, cs_item_sk#33, cs_net_profit#34] + +(42) Filter [codegen id : 13] +Input [4]: [cs_sold_date_sk#31, cs_bill_customer_sk#32, cs_item_sk#33, cs_net_profit#34] +Condition : ((isnotnull(cs_bill_customer_sk#32) AND isnotnull(cs_item_sk#33)) AND isnotnull(cs_sold_date_sk#31)) -(46) Filter [codegen id : 15] -Input [4]: [cs_sold_date_sk#32, cs_bill_customer_sk#33, cs_item_sk#34, cs_net_profit#35] -Condition : ((isnotnull(cs_bill_customer_sk#33) AND isnotnull(cs_item_sk#34)) AND isnotnull(cs_sold_date_sk#32)) +(43) ReusedExchange [Reuses operator id: 35] +Output [1]: [d_date_sk#35] -(47) ReusedExchange [Reuses operator id: 35] -Output [1]: [d_date_sk#36] +(44) BroadcastHashJoin [codegen id : 13] +Left keys [1]: [cs_sold_date_sk#31] +Right keys [1]: [d_date_sk#35] +Join condition: None + +(45) Project [codegen id : 13] +Output [3]: [cs_bill_customer_sk#32, cs_item_sk#33, cs_net_profit#34] +Input [5]: [cs_sold_date_sk#31, cs_bill_customer_sk#32, cs_item_sk#33, cs_net_profit#34, d_date_sk#35] + +(46) Exchange +Input [3]: [cs_bill_customer_sk#32, cs_item_sk#33, cs_net_profit#34] +Arguments: hashpartitioning(cast(cs_bill_customer_sk#32 as bigint), cast(cs_item_sk#33 as bigint), 5), ENSURE_REQUIREMENTS, [id=#36] -(48) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [cs_sold_date_sk#32] -Right keys [1]: [d_date_sk#36] +(47) Sort [codegen id : 14] +Input [3]: [cs_bill_customer_sk#32, cs_item_sk#33, cs_net_profit#34] +Arguments: [cast(cs_bill_customer_sk#32 as bigint) ASC NULLS FIRST, cast(cs_item_sk#33 as bigint) ASC NULLS FIRST], false, 0 + +(48) SortMergeJoin [codegen id : 15] +Left keys [2]: [sr_customer_sk#23, sr_item_sk#22] +Right keys [2]: [cast(cs_bill_customer_sk#32 as bigint), cast(cs_item_sk#33 as bigint)] Join condition: None (49) Project [codegen id : 15] -Output [3]: [cs_bill_customer_sk#33, cs_item_sk#34, cs_net_profit#35] -Input [5]: [cs_sold_date_sk#32, cs_bill_customer_sk#33, cs_item_sk#34, cs_net_profit#35, d_date_sk#36] +Output [5]: [sr_item_sk#22, sr_customer_sk#23, sr_ticket_number#24, sr_net_loss#25, cs_net_profit#34] +Input [7]: [sr_item_sk#22, sr_customer_sk#23, sr_ticket_number#24, sr_net_loss#25, cs_bill_customer_sk#32, cs_item_sk#33, cs_net_profit#34] (50) Exchange -Input [3]: [cs_bill_customer_sk#33, cs_item_sk#34, cs_net_profit#35] -Arguments: hashpartitioning(cast(cs_bill_customer_sk#33 as bigint), cast(cs_item_sk#34 as bigint), 5), true, [id=#37] +Input [5]: [sr_item_sk#22, sr_customer_sk#23, sr_ticket_number#24, sr_net_loss#25, cs_net_profit#34] +Arguments: hashpartitioning(sr_customer_sk#23, sr_item_sk#22, sr_ticket_number#24, 5), ENSURE_REQUIREMENTS, [id=#37] (51) Sort [codegen id : 16] -Input [3]: [cs_bill_customer_sk#33, cs_item_sk#34, cs_net_profit#35] -Arguments: [cast(cs_bill_customer_sk#33 as bigint) ASC NULLS FIRST, cast(cs_item_sk#34 as bigint) ASC NULLS FIRST], false, 0 +Input [5]: [sr_item_sk#22, sr_customer_sk#23, sr_ticket_number#24, sr_net_loss#25, cs_net_profit#34] +Arguments: [sr_customer_sk#23 ASC NULLS FIRST, sr_item_sk#22 ASC NULLS FIRST, sr_ticket_number#24 ASC NULLS FIRST], false, 0 (52) SortMergeJoin [codegen id : 17] -Left keys [2]: [sr_customer_sk#23, sr_item_sk#22] -Right keys [2]: [cast(cs_bill_customer_sk#33 as bigint), cast(cs_item_sk#34 as bigint)] +Left keys [3]: [cast(ss_customer_sk#3 as bigint), cast(ss_item_sk#2 as bigint), cast(ss_ticket_number#5 as bigint)] +Right keys [3]: [sr_customer_sk#23, sr_item_sk#22, sr_ticket_number#24] Join condition: None (53) Project [codegen id : 17] -Output [7]: [ss_net_profit#6, sr_net_loss#25, cs_net_profit#35, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18] -Input [11]: [ss_net_profit#6, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18, sr_item_sk#22, sr_customer_sk#23, sr_net_loss#25, cs_bill_customer_sk#33, cs_item_sk#34, cs_net_profit#35] +Output [7]: [ss_net_profit#6, sr_net_loss#25, cs_net_profit#34, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18] +Input [13]: [ss_item_sk#2, ss_customer_sk#3, ss_ticket_number#5, ss_net_profit#6, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18, sr_item_sk#22, sr_customer_sk#23, sr_ticket_number#24, sr_net_loss#25, cs_net_profit#34] (54) HashAggregate [codegen id : 17] -Input [7]: [ss_net_profit#6, sr_net_loss#25, cs_net_profit#35, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18] +Input [7]: [ss_net_profit#6, sr_net_loss#25, cs_net_profit#34, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18] Keys [4]: [i_item_id#17, i_item_desc#18, s_store_id#12, s_store_name#13] -Functions [3]: [partial_sum(UnscaledValue(ss_net_profit#6)), partial_sum(UnscaledValue(sr_net_loss#25)), partial_sum(UnscaledValue(cs_net_profit#35))] +Functions [3]: [partial_sum(UnscaledValue(ss_net_profit#6)), partial_sum(UnscaledValue(sr_net_loss#25)), partial_sum(UnscaledValue(cs_net_profit#34))] Aggregate Attributes [3]: [sum#38, sum#39, sum#40] Results [7]: [i_item_id#17, i_item_desc#18, s_store_id#12, s_store_name#13, sum#41, sum#42, sum#43] (55) Exchange Input [7]: [i_item_id#17, i_item_desc#18, s_store_id#12, s_store_name#13, sum#41, sum#42, sum#43] -Arguments: hashpartitioning(i_item_id#17, i_item_desc#18, s_store_id#12, s_store_name#13, 5), true, [id=#44] +Arguments: hashpartitioning(i_item_id#17, i_item_desc#18, s_store_id#12, s_store_name#13, 5), ENSURE_REQUIREMENTS, [id=#44] (56) HashAggregate [codegen id : 18] Input [7]: [i_item_id#17, i_item_desc#18, s_store_id#12, s_store_name#13, sum#41, sum#42, sum#43] Keys [4]: [i_item_id#17, i_item_desc#18, s_store_id#12, s_store_name#13] -Functions [3]: [sum(UnscaledValue(ss_net_profit#6)), sum(UnscaledValue(sr_net_loss#25)), sum(UnscaledValue(cs_net_profit#35))] -Aggregate Attributes [3]: [sum(UnscaledValue(ss_net_profit#6))#45, sum(UnscaledValue(sr_net_loss#25))#46, sum(UnscaledValue(cs_net_profit#35))#47] -Results [7]: [i_item_id#17, i_item_desc#18, s_store_id#12, s_store_name#13, MakeDecimal(sum(UnscaledValue(ss_net_profit#6))#45,17,2) AS store_sales_profit#48, MakeDecimal(sum(UnscaledValue(sr_net_loss#25))#46,17,2) AS store_returns_loss#49, MakeDecimal(sum(UnscaledValue(cs_net_profit#35))#47,17,2) AS catalog_sales_profit#50] +Functions [3]: [sum(UnscaledValue(ss_net_profit#6)), sum(UnscaledValue(sr_net_loss#25)), sum(UnscaledValue(cs_net_profit#34))] +Aggregate Attributes [3]: [sum(UnscaledValue(ss_net_profit#6))#45, sum(UnscaledValue(sr_net_loss#25))#46, sum(UnscaledValue(cs_net_profit#34))#47] +Results [7]: [i_item_id#17, i_item_desc#18, s_store_id#12, s_store_name#13, MakeDecimal(sum(UnscaledValue(ss_net_profit#6))#45,17,2) AS store_sales_profit#48, MakeDecimal(sum(UnscaledValue(sr_net_loss#25))#46,17,2) AS store_returns_loss#49, MakeDecimal(sum(UnscaledValue(cs_net_profit#34))#47,17,2) AS catalog_sales_profit#50] (57) TakeOrderedAndProject Input [7]: [i_item_id#17, i_item_desc#18, s_store_id#12, s_store_name#13, store_sales_profit#48, store_returns_loss#49, catalog_sales_profit#50] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.sf100/simplified.txt index ad9fa718ff2bd..9b53cdaa5dc67 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.sf100/simplified.txt @@ -6,67 +6,67 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales WholeStageCodegen (17) HashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,ss_net_profit,sr_net_loss,cs_net_profit] [sum,sum,sum,sum,sum,sum] Project [ss_net_profit,sr_net_loss,cs_net_profit,s_store_id,s_store_name,i_item_id,i_item_desc] - SortMergeJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk] + SortMergeJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number] InputAdapter - WholeStageCodegen (13) - Sort [sr_customer_sk,sr_item_sk] + WholeStageCodegen (8) + Sort [ss_customer_sk,ss_item_sk,ss_ticket_number] InputAdapter - Exchange [sr_customer_sk,sr_item_sk] #2 - WholeStageCodegen (12) - Project [ss_net_profit,s_store_id,s_store_name,i_item_id,i_item_desc,sr_item_sk,sr_customer_sk,sr_net_loss] - SortMergeJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number] + Exchange [ss_customer_sk,ss_item_sk,ss_ticket_number] #2 + WholeStageCodegen (7) + Project [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_net_profit,s_store_id,s_store_name,i_item_id,i_item_desc] + SortMergeJoin [ss_item_sk,i_item_sk] InputAdapter - WholeStageCodegen (8) - Sort [ss_customer_sk,ss_item_sk,ss_ticket_number] + WholeStageCodegen (4) + Sort [ss_item_sk] InputAdapter - Exchange [ss_customer_sk,ss_item_sk,ss_ticket_number] #3 - WholeStageCodegen (7) - Project [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_net_profit,s_store_id,s_store_name,i_item_id,i_item_desc] - SortMergeJoin [ss_item_sk,i_item_sk] - InputAdapter - WholeStageCodegen (4) - Sort [ss_item_sk] - InputAdapter - Exchange [ss_item_sk] #4 - WholeStageCodegen (3) - Project [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_net_profit,s_store_id,s_store_name] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_customer_sk,ss_item_sk,ss_ticket_number,ss_sold_date_sk,ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_profit] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_moy,d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (2) - Filter [s_store_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store [s_store_sk,s_store_id,s_store_name] - InputAdapter - WholeStageCodegen (6) - Sort [i_item_sk] - InputAdapter - Exchange [i_item_sk] #7 - WholeStageCodegen (5) - Filter [i_item_sk] + Exchange [ss_item_sk] #3 + WholeStageCodegen (3) + Project [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_net_profit,s_store_id,s_store_name] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_profit] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_customer_sk,ss_item_sk,ss_ticket_number,ss_sold_date_sk,ss_store_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_profit] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (1) + Project [d_date_sk] + Filter [d_moy,d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_item_sk,i_item_id,i_item_desc] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (2) + Filter [s_store_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store [s_store_sk,s_store_id,s_store_name] + InputAdapter + WholeStageCodegen (6) + Sort [i_item_sk] + InputAdapter + Exchange [i_item_sk] #6 + WholeStageCodegen (5) + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_item_id,i_item_desc] + InputAdapter + WholeStageCodegen (16) + Sort [sr_customer_sk,sr_item_sk,sr_ticket_number] + InputAdapter + Exchange [sr_customer_sk,sr_item_sk,sr_ticket_number] #7 + WholeStageCodegen (15) + Project [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,cs_net_profit] + SortMergeJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk] InputAdapter WholeStageCodegen (11) - Sort [sr_customer_sk,sr_item_sk,sr_ticket_number] + Sort [sr_customer_sk,sr_item_sk] InputAdapter - Exchange [sr_customer_sk,sr_item_sk,sr_ticket_number] #8 + Exchange [sr_customer_sk,sr_item_sk] #8 WholeStageCodegen (10) Project [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss] BroadcastHashJoin [sr_returned_date_sk,d_date_sk] @@ -82,17 +82,17 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - WholeStageCodegen (16) - Sort [cs_bill_customer_sk,cs_item_sk] - InputAdapter - Exchange [cs_bill_customer_sk,cs_item_sk] #10 - WholeStageCodegen (15) - Project [cs_bill_customer_sk,cs_item_sk,cs_net_profit] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk,cs_net_profit] InputAdapter - ReusedExchange [d_date_sk] #9 + WholeStageCodegen (14) + Sort [cs_bill_customer_sk,cs_item_sk] + InputAdapter + Exchange [cs_bill_customer_sk,cs_item_sk] #10 + WholeStageCodegen (13) + Project [cs_bill_customer_sk,cs_item_sk,cs_net_profit] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Filter [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk,cs_net_profit] + InputAdapter + ReusedExchange [d_date_sk] #9 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 8185680b58670..cb8522545f1d3 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 @@ -9,8 +9,8 @@ TakeOrderedAndProject (67) : +- * HashAggregate (30) : +- * Project (29) : +- * BroadcastHashJoin Inner BuildRight (28) - : :- * Project (22) - : : +- * BroadcastHashJoin Inner BuildRight (21) + : :- * Project (17) + : : +- * BroadcastHashJoin Inner BuildRight (16) : : :- * Project (10) : : : +- * BroadcastHashJoin Inner BuildRight (9) : : : :- * Filter (3) @@ -21,21 +21,21 @@ TakeOrderedAndProject (67) : : : +- * Filter (6) : : : +- * ColumnarToRow (5) : : : +- Scan parquet default.date_dim (4) - : : +- BroadcastExchange (20) - : : +- * BroadcastHashJoin LeftSemi BuildRight (19) - : : :- * Filter (13) - : : : +- * ColumnarToRow (12) - : : : +- Scan parquet default.item (11) - : : +- BroadcastExchange (18) - : : +- * Project (17) - : : +- * Filter (16) - : : +- * ColumnarToRow (15) - : : +- Scan parquet default.item (14) + : : +- BroadcastExchange (15) + : : +- * Project (14) + : : +- * Filter (13) + : : +- * ColumnarToRow (12) + : : +- Scan parquet default.customer_address (11) : +- BroadcastExchange (27) - : +- * Project (26) - : +- * Filter (25) - : +- * ColumnarToRow (24) - : +- Scan parquet default.customer_address (23) + : +- * BroadcastHashJoin LeftSemi BuildRight (26) + : :- * Filter (20) + : : +- * ColumnarToRow (19) + : : +- Scan parquet default.item (18) + : +- BroadcastExchange (25) + : +- * Project (24) + : +- * Filter (23) + : +- * ColumnarToRow (22) + : +- Scan parquet default.item (21) :- * HashAggregate (47) : +- Exchange (46) : +- * HashAggregate (45) @@ -113,108 +113,108 @@ Join condition: None Output [3]: [ss_item_sk#2, ss_addr_sk#3, ss_ext_sales_price#4] Input [5]: [ss_sold_date_sk#1, ss_item_sk#2, ss_addr_sk#3, ss_ext_sales_price#4, d_date_sk#5] -(11) Scan parquet default.item -Output [2]: [i_item_sk#9, i_manufact_id#10] +(11) Scan parquet default.customer_address +Output [2]: [ca_address_sk#9, ca_gmt_offset#10] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer_address] +PushedFilters: [IsNotNull(ca_gmt_offset), EqualTo(ca_gmt_offset,-5.00), IsNotNull(ca_address_sk)] +ReadSchema: struct + +(12) ColumnarToRow [codegen id : 2] +Input [2]: [ca_address_sk#9, ca_gmt_offset#10] + +(13) Filter [codegen id : 2] +Input [2]: [ca_address_sk#9, ca_gmt_offset#10] +Condition : ((isnotnull(ca_gmt_offset#10) AND (ca_gmt_offset#10 = -5.00)) AND isnotnull(ca_address_sk#9)) + +(14) Project [codegen id : 2] +Output [1]: [ca_address_sk#9] +Input [2]: [ca_address_sk#9, ca_gmt_offset#10] + +(15) BroadcastExchange +Input [1]: [ca_address_sk#9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#11] + +(16) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [ss_addr_sk#3] +Right keys [1]: [ca_address_sk#9] +Join condition: None + +(17) Project [codegen id : 5] +Output [2]: [ss_item_sk#2, ss_ext_sales_price#4] +Input [4]: [ss_item_sk#2, ss_addr_sk#3, ss_ext_sales_price#4, ca_address_sk#9] + +(18) Scan parquet default.item +Output [2]: [i_item_sk#12, i_manufact_id#13] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(12) ColumnarToRow [codegen id : 3] -Input [2]: [i_item_sk#9, i_manufact_id#10] +(19) ColumnarToRow [codegen id : 4] +Input [2]: [i_item_sk#12, i_manufact_id#13] -(13) Filter [codegen id : 3] -Input [2]: [i_item_sk#9, i_manufact_id#10] -Condition : isnotnull(i_item_sk#9) +(20) Filter [codegen id : 4] +Input [2]: [i_item_sk#12, i_manufact_id#13] +Condition : isnotnull(i_item_sk#12) -(14) Scan parquet default.item -Output [2]: [i_category#11, i_manufact_id#10] +(21) Scan parquet default.item +Output [2]: [i_category#14, i_manufact_id#13] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Electronics)] ReadSchema: struct -(15) ColumnarToRow [codegen id : 2] -Input [2]: [i_category#11, i_manufact_id#10] +(22) ColumnarToRow [codegen id : 3] +Input [2]: [i_category#14, i_manufact_id#13] -(16) Filter [codegen id : 2] -Input [2]: [i_category#11, i_manufact_id#10] -Condition : (isnotnull(i_category#11) AND (i_category#11 = Electronics)) +(23) Filter [codegen id : 3] +Input [2]: [i_category#14, i_manufact_id#13] +Condition : (isnotnull(i_category#14) AND (i_category#14 = Electronics)) -(17) Project [codegen id : 2] -Output [1]: [i_manufact_id#10 AS i_manufact_id#10#12] -Input [2]: [i_category#11, i_manufact_id#10] +(24) Project [codegen id : 3] +Output [1]: [i_manufact_id#13 AS i_manufact_id#13#15] +Input [2]: [i_category#14, i_manufact_id#13] -(18) BroadcastExchange -Input [1]: [i_manufact_id#10#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#13] +(25) BroadcastExchange +Input [1]: [i_manufact_id#13#15] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#16] -(19) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [i_manufact_id#10] -Right keys [1]: [i_manufact_id#10#12] +(26) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [i_manufact_id#13] +Right keys [1]: [i_manufact_id#13#15] Join condition: None -(20) BroadcastExchange -Input [2]: [i_item_sk#9, i_manufact_id#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#14] - -(21) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_item_sk#2] -Right keys [1]: [i_item_sk#9] -Join condition: None - -(22) Project [codegen id : 5] -Output [3]: [ss_addr_sk#3, ss_ext_sales_price#4, i_manufact_id#10] -Input [5]: [ss_item_sk#2, ss_addr_sk#3, ss_ext_sales_price#4, i_item_sk#9, i_manufact_id#10] - -(23) Scan parquet default.customer_address -Output [2]: [ca_address_sk#15, ca_gmt_offset#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_gmt_offset), EqualTo(ca_gmt_offset,-5.00), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(24) ColumnarToRow [codegen id : 4] -Input [2]: [ca_address_sk#15, ca_gmt_offset#16] - -(25) Filter [codegen id : 4] -Input [2]: [ca_address_sk#15, ca_gmt_offset#16] -Condition : ((isnotnull(ca_gmt_offset#16) AND (ca_gmt_offset#16 = -5.00)) AND isnotnull(ca_address_sk#15)) - -(26) Project [codegen id : 4] -Output [1]: [ca_address_sk#15] -Input [2]: [ca_address_sk#15, ca_gmt_offset#16] - (27) BroadcastExchange -Input [1]: [ca_address_sk#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#17] +Input [2]: [i_item_sk#12, i_manufact_id#13] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#17] (28) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_addr_sk#3] -Right keys [1]: [ca_address_sk#15] +Left keys [1]: [ss_item_sk#2] +Right keys [1]: [i_item_sk#12] Join condition: None (29) Project [codegen id : 5] -Output [2]: [ss_ext_sales_price#4, i_manufact_id#10] -Input [4]: [ss_addr_sk#3, ss_ext_sales_price#4, i_manufact_id#10, ca_address_sk#15] +Output [2]: [ss_ext_sales_price#4, i_manufact_id#13] +Input [4]: [ss_item_sk#2, ss_ext_sales_price#4, i_item_sk#12, i_manufact_id#13] (30) HashAggregate [codegen id : 5] -Input [2]: [ss_ext_sales_price#4, i_manufact_id#10] -Keys [1]: [i_manufact_id#10] +Input [2]: [ss_ext_sales_price#4, i_manufact_id#13] +Keys [1]: [i_manufact_id#13] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#4))] Aggregate Attributes [1]: [sum#18] -Results [2]: [i_manufact_id#10, sum#19] +Results [2]: [i_manufact_id#13, sum#19] (31) Exchange -Input [2]: [i_manufact_id#10, sum#19] -Arguments: hashpartitioning(i_manufact_id#10, 5), true, [id=#20] +Input [2]: [i_manufact_id#13, sum#19] +Arguments: hashpartitioning(i_manufact_id#13, 5), ENSURE_REQUIREMENTS, [id=#20] (32) HashAggregate [codegen id : 6] -Input [2]: [i_manufact_id#10, sum#19] -Keys [1]: [i_manufact_id#10] +Input [2]: [i_manufact_id#13, sum#19] +Keys [1]: [i_manufact_id#13] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#4))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#4))#21] -Results [2]: [i_manufact_id#10, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#4))#21,17,2) AS total_sales#22] +Results [2]: [i_manufact_id#13, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#4))#21,17,2) AS total_sales#22] (33) Scan parquet default.catalog_sales Output [4]: [cs_sold_date_sk#23, cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26] @@ -242,47 +242,47 @@ Join condition: None Output [3]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26] Input [5]: [cs_sold_date_sk#23, cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, d_date_sk#5] -(39) ReusedExchange [Reuses operator id: 20] -Output [2]: [i_item_sk#9, i_manufact_id#10] +(39) ReusedExchange [Reuses operator id: 15] +Output [1]: [ca_address_sk#9] (40) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [cs_item_sk#25] -Right keys [1]: [i_item_sk#9] +Left keys [1]: [cs_bill_addr_sk#24] +Right keys [1]: [ca_address_sk#9] Join condition: None (41) Project [codegen id : 11] -Output [3]: [cs_bill_addr_sk#24, cs_ext_sales_price#26, i_manufact_id#10] -Input [5]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, i_item_sk#9, i_manufact_id#10] +Output [2]: [cs_item_sk#25, cs_ext_sales_price#26] +Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, ca_address_sk#9] (42) ReusedExchange [Reuses operator id: 27] -Output [1]: [ca_address_sk#15] +Output [2]: [i_item_sk#12, i_manufact_id#13] (43) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [cs_bill_addr_sk#24] -Right keys [1]: [ca_address_sk#15] +Left keys [1]: [cs_item_sk#25] +Right keys [1]: [i_item_sk#12] Join condition: None (44) Project [codegen id : 11] -Output [2]: [cs_ext_sales_price#26, i_manufact_id#10] -Input [4]: [cs_bill_addr_sk#24, cs_ext_sales_price#26, i_manufact_id#10, ca_address_sk#15] +Output [2]: [cs_ext_sales_price#26, i_manufact_id#13] +Input [4]: [cs_item_sk#25, cs_ext_sales_price#26, i_item_sk#12, i_manufact_id#13] (45) HashAggregate [codegen id : 11] -Input [2]: [cs_ext_sales_price#26, i_manufact_id#10] -Keys [1]: [i_manufact_id#10] +Input [2]: [cs_ext_sales_price#26, i_manufact_id#13] +Keys [1]: [i_manufact_id#13] Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#26))] Aggregate Attributes [1]: [sum#27] -Results [2]: [i_manufact_id#10, sum#28] +Results [2]: [i_manufact_id#13, sum#28] (46) Exchange -Input [2]: [i_manufact_id#10, sum#28] -Arguments: hashpartitioning(i_manufact_id#10, 5), true, [id=#29] +Input [2]: [i_manufact_id#13, sum#28] +Arguments: hashpartitioning(i_manufact_id#13, 5), ENSURE_REQUIREMENTS, [id=#29] (47) HashAggregate [codegen id : 12] -Input [2]: [i_manufact_id#10, sum#28] -Keys [1]: [i_manufact_id#10] +Input [2]: [i_manufact_id#13, sum#28] +Keys [1]: [i_manufact_id#13] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#26))] Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#26))#30] -Results [2]: [i_manufact_id#10, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#26))#30,17,2) AS total_sales#31] +Results [2]: [i_manufact_id#13, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#26))#30,17,2) AS total_sales#31] (48) Scan parquet default.web_sales Output [4]: [ws_sold_date_sk#32, ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35] @@ -310,69 +310,69 @@ Join condition: None Output [3]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35] Input [5]: [ws_sold_date_sk#32, ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, d_date_sk#5] -(54) ReusedExchange [Reuses operator id: 20] -Output [2]: [i_item_sk#9, i_manufact_id#10] +(54) ReusedExchange [Reuses operator id: 15] +Output [1]: [ca_address_sk#9] (55) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_item_sk#33] -Right keys [1]: [i_item_sk#9] +Left keys [1]: [ws_bill_addr_sk#34] +Right keys [1]: [ca_address_sk#9] Join condition: None (56) Project [codegen id : 17] -Output [3]: [ws_bill_addr_sk#34, ws_ext_sales_price#35, i_manufact_id#10] -Input [5]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, i_item_sk#9, i_manufact_id#10] +Output [2]: [ws_item_sk#33, ws_ext_sales_price#35] +Input [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ca_address_sk#9] (57) ReusedExchange [Reuses operator id: 27] -Output [1]: [ca_address_sk#15] +Output [2]: [i_item_sk#12, i_manufact_id#13] (58) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_bill_addr_sk#34] -Right keys [1]: [ca_address_sk#15] +Left keys [1]: [ws_item_sk#33] +Right keys [1]: [i_item_sk#12] Join condition: None (59) Project [codegen id : 17] -Output [2]: [ws_ext_sales_price#35, i_manufact_id#10] -Input [4]: [ws_bill_addr_sk#34, ws_ext_sales_price#35, i_manufact_id#10, ca_address_sk#15] +Output [2]: [ws_ext_sales_price#35, i_manufact_id#13] +Input [4]: [ws_item_sk#33, ws_ext_sales_price#35, i_item_sk#12, i_manufact_id#13] (60) HashAggregate [codegen id : 17] -Input [2]: [ws_ext_sales_price#35, i_manufact_id#10] -Keys [1]: [i_manufact_id#10] +Input [2]: [ws_ext_sales_price#35, i_manufact_id#13] +Keys [1]: [i_manufact_id#13] Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#35))] Aggregate Attributes [1]: [sum#36] -Results [2]: [i_manufact_id#10, sum#37] +Results [2]: [i_manufact_id#13, sum#37] (61) Exchange -Input [2]: [i_manufact_id#10, sum#37] -Arguments: hashpartitioning(i_manufact_id#10, 5), true, [id=#38] +Input [2]: [i_manufact_id#13, sum#37] +Arguments: hashpartitioning(i_manufact_id#13, 5), ENSURE_REQUIREMENTS, [id=#38] (62) HashAggregate [codegen id : 18] -Input [2]: [i_manufact_id#10, sum#37] -Keys [1]: [i_manufact_id#10] +Input [2]: [i_manufact_id#13, sum#37] +Keys [1]: [i_manufact_id#13] Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#35))] Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#35))#39] -Results [2]: [i_manufact_id#10, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#35))#39,17,2) AS total_sales#40] +Results [2]: [i_manufact_id#13, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#35))#39,17,2) AS total_sales#40] (63) Union (64) HashAggregate [codegen id : 19] -Input [2]: [i_manufact_id#10, total_sales#22] -Keys [1]: [i_manufact_id#10] +Input [2]: [i_manufact_id#13, total_sales#22] +Keys [1]: [i_manufact_id#13] Functions [1]: [partial_sum(total_sales#22)] Aggregate Attributes [2]: [sum#41, isEmpty#42] -Results [3]: [i_manufact_id#10, sum#43, isEmpty#44] +Results [3]: [i_manufact_id#13, sum#43, isEmpty#44] (65) Exchange -Input [3]: [i_manufact_id#10, sum#43, isEmpty#44] -Arguments: hashpartitioning(i_manufact_id#10, 5), true, [id=#45] +Input [3]: [i_manufact_id#13, sum#43, isEmpty#44] +Arguments: hashpartitioning(i_manufact_id#13, 5), ENSURE_REQUIREMENTS, [id=#45] (66) HashAggregate [codegen id : 20] -Input [3]: [i_manufact_id#10, sum#43, isEmpty#44] -Keys [1]: [i_manufact_id#10] +Input [3]: [i_manufact_id#13, sum#43, isEmpty#44] +Keys [1]: [i_manufact_id#13] Functions [1]: [sum(total_sales#22)] Aggregate Attributes [1]: [sum(total_sales#22)#46] -Results [2]: [i_manufact_id#10, sum(total_sales#22)#46 AS total_sales#47] +Results [2]: [i_manufact_id#13, sum(total_sales#22)#46 AS total_sales#47] (67) TakeOrderedAndProject -Input [2]: [i_manufact_id#10, total_sales#47] -Arguments: 100, [total_sales#47 ASC NULLS FIRST], [i_manufact_id#10, total_sales#47] +Input [2]: [i_manufact_id#13, total_sales#47] +Arguments: 100, [total_sales#47 ASC NULLS FIRST], [i_manufact_id#13, total_sales#47] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/simplified.txt index 410def2466e1a..14787f0bbce7b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/simplified.txt @@ -14,9 +14,9 @@ TakeOrderedAndProject [total_sales,i_manufact_id] WholeStageCodegen (5) HashAggregate [i_manufact_id,ss_ext_sales_price] [sum,sum] Project [ss_ext_sales_price,i_manufact_id] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_addr_sk,ss_ext_sales_price,i_manufact_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_ext_sales_price] + BroadcastHashJoin [ss_addr_sk,ca_address_sk] Project [ss_item_sk,ss_addr_sk,ss_ext_sales_price] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Filter [ss_sold_date_sk,ss_addr_sk,ss_item_sk] @@ -33,28 +33,28 @@ TakeOrderedAndProject [total_sales,i_manufact_id] Scan parquet default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #4 - WholeStageCodegen (3) - BroadcastHashJoin [i_manufact_id,i_manufact_id] - Filter [i_item_sk] + WholeStageCodegen (2) + Project [ca_address_sk] + Filter [ca_gmt_offset,ca_address_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_item_sk,i_manufact_id] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - Project [i_manufact_id] - Filter [i_category] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_category,i_manufact_id] + Scan parquet default.customer_address [ca_address_sk,ca_gmt_offset] InputAdapter - BroadcastExchange #6 + BroadcastExchange #5 WholeStageCodegen (4) - Project [ca_address_sk] - Filter [ca_gmt_offset,ca_address_sk] + BroadcastHashJoin [i_manufact_id,i_manufact_id] + Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_gmt_offset] + Scan parquet default.item [i_item_sk,i_manufact_id] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (3) + Project [i_manufact_id] + Filter [i_category] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_category,i_manufact_id] WholeStageCodegen (12) HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] InputAdapter @@ -62,9 +62,9 @@ TakeOrderedAndProject [total_sales,i_manufact_id] WholeStageCodegen (11) HashAggregate [i_manufact_id,cs_ext_sales_price] [sum,sum] Project [cs_ext_sales_price,i_manufact_id] - BroadcastHashJoin [cs_bill_addr_sk,ca_address_sk] - Project [cs_bill_addr_sk,cs_ext_sales_price,i_manufact_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk,cs_ext_sales_price] + BroadcastHashJoin [cs_bill_addr_sk,ca_address_sk] Project [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] Filter [cs_sold_date_sk,cs_bill_addr_sk,cs_item_sk] @@ -74,9 +74,9 @@ TakeOrderedAndProject [total_sales,i_manufact_id] InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter - ReusedExchange [i_item_sk,i_manufact_id] #4 + ReusedExchange [ca_address_sk] #4 InputAdapter - ReusedExchange [ca_address_sk] #6 + ReusedExchange [i_item_sk,i_manufact_id] #5 WholeStageCodegen (18) HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),total_sales,sum] InputAdapter @@ -84,9 +84,9 @@ TakeOrderedAndProject [total_sales,i_manufact_id] WholeStageCodegen (17) HashAggregate [i_manufact_id,ws_ext_sales_price] [sum,sum] Project [ws_ext_sales_price,i_manufact_id] - BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] - Project [ws_bill_addr_sk,ws_ext_sales_price,i_manufact_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Project [ws_item_sk,ws_ext_sales_price] + BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] Project [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Filter [ws_sold_date_sk,ws_bill_addr_sk,ws_item_sk] @@ -96,6 +96,6 @@ TakeOrderedAndProject [total_sales,i_manufact_id] InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter - ReusedExchange [i_item_sk,i_manufact_id] #4 + ReusedExchange [ca_address_sk] #4 InputAdapter - ReusedExchange [ca_address_sk] #6 + ReusedExchange [i_item_sk,i_manufact_id] #5 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 d7a8c103285cb..6492918d3aa13 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 @@ -6,117 +6,117 @@ TakeOrderedAndProject (21) +- * Project (17) +- * BroadcastHashJoin Inner BuildRight (16) :- * Project (10) - : +- * BroadcastHashJoin Inner BuildRight (9) - : :- * Filter (3) - : : +- * ColumnarToRow (2) - : : +- Scan parquet default.store_sales (1) - : +- BroadcastExchange (8) - : +- * Project (7) - : +- * Filter (6) - : +- * ColumnarToRow (5) - : +- Scan parquet default.item (4) + : +- * BroadcastHashJoin Inner BuildLeft (9) + : :- BroadcastExchange (5) + : : +- * Project (4) + : : +- * Filter (3) + : : +- * ColumnarToRow (2) + : : +- Scan parquet default.date_dim (1) + : +- * Filter (8) + : +- * ColumnarToRow (7) + : +- Scan parquet default.store_sales (6) +- BroadcastExchange (15) +- * Project (14) +- * Filter (13) +- * ColumnarToRow (12) - +- Scan parquet default.date_dim (11) + +- Scan parquet default.item (11) -(1) Scan parquet default.store_sales -Output [3]: [ss_sold_date_sk#1, ss_item_sk#2, ss_ext_sales_price#3] +(1) Scan parquet default.date_dim +Output [3]: [d_date_sk#1, d_year#2, d_moy#3] Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_sold_date_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,11), EqualTo(d_year,2000), IsNotNull(d_date_sk)] +ReadSchema: struct -(2) ColumnarToRow [codegen id : 3] -Input [3]: [ss_sold_date_sk#1, ss_item_sk#2, ss_ext_sales_price#3] +(2) ColumnarToRow [codegen id : 1] +Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -(3) Filter [codegen id : 3] -Input [3]: [ss_sold_date_sk#1, ss_item_sk#2, ss_ext_sales_price#3] -Condition : (isnotnull(ss_sold_date_sk#1) AND isnotnull(ss_item_sk#2)) +(3) Filter [codegen id : 1] +Input [3]: [d_date_sk#1, d_year#2, d_moy#3] +Condition : ((((isnotnull(d_moy#3) AND isnotnull(d_year#2)) AND (d_moy#3 = 11)) AND (d_year#2 = 2000)) AND isnotnull(d_date_sk#1)) -(4) Scan parquet default.item -Output [4]: [i_item_sk#4, i_brand_id#5, i_brand#6, i_manager_id#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manager_id), EqualTo(i_manager_id,1), IsNotNull(i_item_sk)] -ReadSchema: struct +(4) Project [codegen id : 1] +Output [2]: [d_date_sk#1, d_year#2] +Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -(5) ColumnarToRow [codegen id : 1] -Input [4]: [i_item_sk#4, i_brand_id#5, i_brand#6, i_manager_id#7] +(5) BroadcastExchange +Input [2]: [d_date_sk#1, d_year#2] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#4] -(6) Filter [codegen id : 1] -Input [4]: [i_item_sk#4, i_brand_id#5, i_brand#6, i_manager_id#7] -Condition : ((isnotnull(i_manager_id#7) AND (i_manager_id#7 = 1)) AND isnotnull(i_item_sk#4)) +(6) Scan parquet default.store_sales +Output [3]: [ss_sold_date_sk#5, ss_item_sk#6, ss_ext_sales_price#7] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store_sales] +PushedFilters: [IsNotNull(ss_sold_date_sk), IsNotNull(ss_item_sk)] +ReadSchema: struct -(7) Project [codegen id : 1] -Output [3]: [i_item_sk#4, i_brand_id#5, i_brand#6] -Input [4]: [i_item_sk#4, i_brand_id#5, i_brand#6, i_manager_id#7] +(7) ColumnarToRow +Input [3]: [ss_sold_date_sk#5, ss_item_sk#6, ss_ext_sales_price#7] -(8) BroadcastExchange -Input [3]: [i_item_sk#4, i_brand_id#5, i_brand#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#8] +(8) Filter +Input [3]: [ss_sold_date_sk#5, ss_item_sk#6, ss_ext_sales_price#7] +Condition : (isnotnull(ss_sold_date_sk#5) AND isnotnull(ss_item_sk#6)) (9) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_item_sk#2] -Right keys [1]: [i_item_sk#4] +Left keys [1]: [d_date_sk#1] +Right keys [1]: [ss_sold_date_sk#5] Join condition: None (10) Project [codegen id : 3] -Output [4]: [ss_sold_date_sk#1, ss_ext_sales_price#3, i_brand_id#5, i_brand#6] -Input [6]: [ss_sold_date_sk#1, ss_item_sk#2, ss_ext_sales_price#3, i_item_sk#4, i_brand_id#5, i_brand#6] +Output [3]: [d_year#2, ss_item_sk#6, ss_ext_sales_price#7] +Input [5]: [d_date_sk#1, d_year#2, ss_sold_date_sk#5, ss_item_sk#6, ss_ext_sales_price#7] -(11) Scan parquet default.date_dim -Output [3]: [d_date_sk#9, d_year#10, d_moy#11] +(11) Scan parquet default.item +Output [4]: [i_item_sk#8, i_brand_id#9, i_brand#10, i_manager_id#11] Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,11), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_manager_id), EqualTo(i_manager_id,1), IsNotNull(i_item_sk)] +ReadSchema: struct (12) ColumnarToRow [codegen id : 2] -Input [3]: [d_date_sk#9, d_year#10, d_moy#11] +Input [4]: [i_item_sk#8, i_brand_id#9, i_brand#10, i_manager_id#11] (13) Filter [codegen id : 2] -Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Condition : ((((isnotnull(d_moy#11) AND isnotnull(d_year#10)) AND (d_moy#11 = 11)) AND (d_year#10 = 2000)) AND isnotnull(d_date_sk#9)) +Input [4]: [i_item_sk#8, i_brand_id#9, i_brand#10, i_manager_id#11] +Condition : ((isnotnull(i_manager_id#11) AND (i_manager_id#11 = 1)) AND isnotnull(i_item_sk#8)) (14) Project [codegen id : 2] -Output [2]: [d_date_sk#9, d_year#10] -Input [3]: [d_date_sk#9, d_year#10, d_moy#11] +Output [3]: [i_item_sk#8, i_brand_id#9, i_brand#10] +Input [4]: [i_item_sk#8, i_brand_id#9, i_brand#10, i_manager_id#11] (15) BroadcastExchange -Input [2]: [d_date_sk#9, d_year#10] +Input [3]: [i_item_sk#8, i_brand_id#9, i_brand#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#12] (16) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#1] -Right keys [1]: [d_date_sk#9] +Left keys [1]: [ss_item_sk#6] +Right keys [1]: [i_item_sk#8] Join condition: None (17) Project [codegen id : 3] -Output [4]: [d_year#10, ss_ext_sales_price#3, i_brand_id#5, i_brand#6] -Input [6]: [ss_sold_date_sk#1, ss_ext_sales_price#3, i_brand_id#5, i_brand#6, d_date_sk#9, d_year#10] +Output [4]: [d_year#2, ss_ext_sales_price#7, i_brand_id#9, i_brand#10] +Input [6]: [d_year#2, ss_item_sk#6, ss_ext_sales_price#7, i_item_sk#8, i_brand_id#9, i_brand#10] (18) HashAggregate [codegen id : 3] -Input [4]: [d_year#10, ss_ext_sales_price#3, i_brand_id#5, i_brand#6] -Keys [3]: [d_year#10, i_brand#6, i_brand_id#5] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#3))] +Input [4]: [d_year#2, ss_ext_sales_price#7, i_brand_id#9, i_brand#10] +Keys [3]: [d_year#2, i_brand#10, i_brand_id#9] +Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#7))] Aggregate Attributes [1]: [sum#13] -Results [4]: [d_year#10, i_brand#6, i_brand_id#5, sum#14] +Results [4]: [d_year#2, i_brand#10, i_brand_id#9, sum#14] (19) Exchange -Input [4]: [d_year#10, i_brand#6, i_brand_id#5, sum#14] -Arguments: hashpartitioning(d_year#10, i_brand#6, i_brand_id#5, 5), true, [id=#15] +Input [4]: [d_year#2, i_brand#10, i_brand_id#9, sum#14] +Arguments: hashpartitioning(d_year#2, i_brand#10, i_brand_id#9, 5), ENSURE_REQUIREMENTS, [id=#15] (20) HashAggregate [codegen id : 4] -Input [4]: [d_year#10, i_brand#6, i_brand_id#5, sum#14] -Keys [3]: [d_year#10, i_brand#6, i_brand_id#5] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#3))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#3))#16] -Results [4]: [d_year#10, i_brand_id#5 AS brand_id#17, i_brand#6 AS brand#18, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#16,17,2) AS ext_price#19] +Input [4]: [d_year#2, i_brand#10, i_brand_id#9, sum#14] +Keys [3]: [d_year#2, i_brand#10, i_brand_id#9] +Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#7))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#7))#16] +Results [4]: [d_year#2, i_brand_id#9 AS brand_id#17, i_brand#10 AS brand#18, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#7))#16,17,2) AS ext_price#19] (21) TakeOrderedAndProject -Input [4]: [d_year#10, brand_id#17, brand#18, ext_price#19] -Arguments: 100, [d_year#10 ASC NULLS FIRST, ext_price#19 DESC NULLS LAST, brand_id#17 ASC NULLS FIRST], [d_year#10, brand_id#17, brand#18, ext_price#19] +Input [4]: [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/q52.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52.sf100/simplified.txt index 8ed500d84390c..f4aaf3df75135 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52.sf100/simplified.txt @@ -6,26 +6,26 @@ TakeOrderedAndProject [d_year,ext_price,brand_id,brand] WholeStageCodegen (3) HashAggregate [d_year,i_brand,i_brand_id,ss_ext_sales_price] [sum,sum] Project [d_year,ss_ext_sales_price,i_brand_id,i_brand] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,ss_ext_sales_price,i_brand_id,i_brand] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_sold_date_sk,ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_ext_sales_price] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [d_year,ss_item_sk,ss_ext_sales_price] + BroadcastHashJoin [d_date_sk,ss_sold_date_sk] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) - Project [i_item_sk,i_brand_id,i_brand] - Filter [i_manager_id,i_item_sk] + Project [d_date_sk,d_year] + Filter [d_moy,d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] + Filter [ss_sold_date_sk,ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_ext_sales_price] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) - Project [d_date_sk,d_year] - Filter [d_moy,d_year,d_date_sk] + Project [i_item_sk,i_brand_id,i_brand] + Filter [i_manager_id,i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year,d_moy] + Scan parquet default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] 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 a1257cd292e48..b8d8aa358d532 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 @@ -6,115 +6,115 @@ TakeOrderedAndProject (21) +- * Project (17) +- * BroadcastHashJoin Inner BuildRight (16) :- * Project (10) - : +- * BroadcastHashJoin Inner BuildRight (9) - : :- * Filter (3) - : : +- * ColumnarToRow (2) - : : +- Scan parquet default.store_sales (1) - : +- BroadcastExchange (8) - : +- * Project (7) - : +- * Filter (6) - : +- * ColumnarToRow (5) - : +- Scan parquet default.item (4) + : +- * BroadcastHashJoin Inner BuildLeft (9) + : :- BroadcastExchange (5) + : : +- * Project (4) + : : +- * Filter (3) + : : +- * ColumnarToRow (2) + : : +- Scan parquet default.date_dim (1) + : +- * Filter (8) + : +- * ColumnarToRow (7) + : +- Scan parquet default.store_sales (6) +- BroadcastExchange (15) +- * Project (14) +- * Filter (13) +- * ColumnarToRow (12) - +- Scan parquet default.date_dim (11) + +- Scan parquet default.item (11) -(1) Scan parquet default.store_sales -Output [3]: [ss_sold_date_sk#1, ss_item_sk#2, ss_ext_sales_price#3] +(1) Scan parquet default.date_dim +Output [3]: [d_date_sk#1, d_year#2, d_moy#3] Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_sold_date_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,11), EqualTo(d_year,1999), IsNotNull(d_date_sk)] +ReadSchema: struct -(2) ColumnarToRow [codegen id : 3] -Input [3]: [ss_sold_date_sk#1, ss_item_sk#2, ss_ext_sales_price#3] +(2) ColumnarToRow [codegen id : 1] +Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -(3) Filter [codegen id : 3] -Input [3]: [ss_sold_date_sk#1, ss_item_sk#2, ss_ext_sales_price#3] -Condition : (isnotnull(ss_sold_date_sk#1) AND isnotnull(ss_item_sk#2)) +(3) Filter [codegen id : 1] +Input [3]: [d_date_sk#1, d_year#2, d_moy#3] +Condition : ((((isnotnull(d_moy#3) AND isnotnull(d_year#2)) AND (d_moy#3 = 11)) AND (d_year#2 = 1999)) AND isnotnull(d_date_sk#1)) -(4) Scan parquet default.item -Output [4]: [i_item_sk#4, i_brand_id#5, i_brand#6, i_manager_id#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manager_id), EqualTo(i_manager_id,28), IsNotNull(i_item_sk)] -ReadSchema: struct +(4) Project [codegen id : 1] +Output [1]: [d_date_sk#1] +Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -(5) ColumnarToRow [codegen id : 1] -Input [4]: [i_item_sk#4, i_brand_id#5, i_brand#6, i_manager_id#7] +(5) BroadcastExchange +Input [1]: [d_date_sk#1] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#4] -(6) Filter [codegen id : 1] -Input [4]: [i_item_sk#4, i_brand_id#5, i_brand#6, i_manager_id#7] -Condition : ((isnotnull(i_manager_id#7) AND (i_manager_id#7 = 28)) AND isnotnull(i_item_sk#4)) +(6) Scan parquet default.store_sales +Output [3]: [ss_sold_date_sk#5, ss_item_sk#6, ss_ext_sales_price#7] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store_sales] +PushedFilters: [IsNotNull(ss_sold_date_sk), IsNotNull(ss_item_sk)] +ReadSchema: struct -(7) Project [codegen id : 1] -Output [3]: [i_item_sk#4, i_brand_id#5, i_brand#6] -Input [4]: [i_item_sk#4, i_brand_id#5, i_brand#6, i_manager_id#7] +(7) ColumnarToRow +Input [3]: [ss_sold_date_sk#5, ss_item_sk#6, ss_ext_sales_price#7] -(8) BroadcastExchange -Input [3]: [i_item_sk#4, i_brand_id#5, i_brand#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#8] +(8) Filter +Input [3]: [ss_sold_date_sk#5, ss_item_sk#6, ss_ext_sales_price#7] +Condition : (isnotnull(ss_sold_date_sk#5) AND isnotnull(ss_item_sk#6)) (9) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_item_sk#2] -Right keys [1]: [i_item_sk#4] +Left keys [1]: [d_date_sk#1] +Right keys [1]: [ss_sold_date_sk#5] Join condition: None (10) Project [codegen id : 3] -Output [4]: [ss_sold_date_sk#1, ss_ext_sales_price#3, i_brand_id#5, i_brand#6] -Input [6]: [ss_sold_date_sk#1, ss_item_sk#2, ss_ext_sales_price#3, i_item_sk#4, i_brand_id#5, i_brand#6] +Output [2]: [ss_item_sk#6, ss_ext_sales_price#7] +Input [4]: [d_date_sk#1, ss_sold_date_sk#5, ss_item_sk#6, ss_ext_sales_price#7] -(11) Scan parquet default.date_dim -Output [3]: [d_date_sk#9, d_year#10, d_moy#11] +(11) Scan parquet default.item +Output [4]: [i_item_sk#8, i_brand_id#9, i_brand#10, i_manager_id#11] Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,11), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_manager_id), EqualTo(i_manager_id,28), IsNotNull(i_item_sk)] +ReadSchema: struct (12) ColumnarToRow [codegen id : 2] -Input [3]: [d_date_sk#9, d_year#10, d_moy#11] +Input [4]: [i_item_sk#8, i_brand_id#9, i_brand#10, i_manager_id#11] (13) Filter [codegen id : 2] -Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Condition : ((((isnotnull(d_moy#11) AND isnotnull(d_year#10)) AND (d_moy#11 = 11)) AND (d_year#10 = 1999)) AND isnotnull(d_date_sk#9)) +Input [4]: [i_item_sk#8, i_brand_id#9, i_brand#10, i_manager_id#11] +Condition : ((isnotnull(i_manager_id#11) AND (i_manager_id#11 = 28)) AND isnotnull(i_item_sk#8)) (14) Project [codegen id : 2] -Output [1]: [d_date_sk#9] -Input [3]: [d_date_sk#9, d_year#10, d_moy#11] +Output [3]: [i_item_sk#8, i_brand_id#9, i_brand#10] +Input [4]: [i_item_sk#8, i_brand_id#9, i_brand#10, i_manager_id#11] (15) BroadcastExchange -Input [1]: [d_date_sk#9] +Input [3]: [i_item_sk#8, i_brand_id#9, i_brand#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#12] (16) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#1] -Right keys [1]: [d_date_sk#9] +Left keys [1]: [ss_item_sk#6] +Right keys [1]: [i_item_sk#8] Join condition: None (17) Project [codegen id : 3] -Output [3]: [ss_ext_sales_price#3, i_brand_id#5, i_brand#6] -Input [5]: [ss_sold_date_sk#1, ss_ext_sales_price#3, i_brand_id#5, i_brand#6, d_date_sk#9] +Output [3]: [ss_ext_sales_price#7, i_brand_id#9, i_brand#10] +Input [5]: [ss_item_sk#6, ss_ext_sales_price#7, i_item_sk#8, i_brand_id#9, i_brand#10] (18) HashAggregate [codegen id : 3] -Input [3]: [ss_ext_sales_price#3, i_brand_id#5, i_brand#6] -Keys [2]: [i_brand#6, i_brand_id#5] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#3))] +Input [3]: [ss_ext_sales_price#7, i_brand_id#9, i_brand#10] +Keys [2]: [i_brand#10, i_brand_id#9] +Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#7))] Aggregate Attributes [1]: [sum#13] -Results [3]: [i_brand#6, i_brand_id#5, sum#14] +Results [3]: [i_brand#10, i_brand_id#9, sum#14] (19) Exchange -Input [3]: [i_brand#6, i_brand_id#5, sum#14] -Arguments: hashpartitioning(i_brand#6, i_brand_id#5, 5), true, [id=#15] +Input [3]: [i_brand#10, i_brand_id#9, sum#14] +Arguments: hashpartitioning(i_brand#10, i_brand_id#9, 5), ENSURE_REQUIREMENTS, [id=#15] (20) HashAggregate [codegen id : 4] -Input [3]: [i_brand#6, i_brand_id#5, sum#14] -Keys [2]: [i_brand#6, i_brand_id#5] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#3))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#3))#16] -Results [3]: [i_brand_id#5 AS brand_id#17, i_brand#6 AS brand#18, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#16,17,2) AS ext_price#19] +Input [3]: [i_brand#10, i_brand_id#9, sum#14] +Keys [2]: [i_brand#10, i_brand_id#9] +Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#7))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#7))#16] +Results [3]: [i_brand_id#9 AS brand_id#17, i_brand#10 AS brand#18, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#7))#16,17,2) AS ext_price#19] (21) TakeOrderedAndProject Input [3]: [brand_id#17, brand#18, ext_price#19] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55.sf100/simplified.txt index b0d0e0d809441..4f375c80678e8 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55.sf100/simplified.txt @@ -6,26 +6,26 @@ TakeOrderedAndProject [ext_price,brand_id,brand] WholeStageCodegen (3) HashAggregate [i_brand,i_brand_id,ss_ext_sales_price] [sum,sum] Project [ss_ext_sales_price,i_brand_id,i_brand] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,ss_ext_sales_price,i_brand_id,i_brand] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_sold_date_sk,ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_ext_sales_price] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_ext_sales_price] + BroadcastHashJoin [d_date_sk,ss_sold_date_sk] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) - Project [i_item_sk,i_brand_id,i_brand] - Filter [i_manager_id,i_item_sk] + Project [d_date_sk] + Filter [d_moy,d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] + Filter [ss_sold_date_sk,ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_ext_sales_price] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) - Project [d_date_sk] - Filter [d_moy,d_year,d_date_sk] + Project [i_item_sk,i_brand_id,i_brand] + Filter [i_manager_id,i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year,d_moy] + Scan parquet default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] 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 3f8106c96379a..3007b11a1a860 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 @@ -11,60 +11,60 @@ TakeOrderedAndProject (79) : +- * BroadcastHashJoin LeftOuter BuildRight (65) : :- * Project (60) : : +- * SortMergeJoin Inner (59) - : : :- * Sort (47) - : : : +- Exchange (46) - : : : +- * Project (45) - : : : +- * BroadcastHashJoin Inner BuildRight (44) - : : : :- * Project (32) - : : : : +- * SortMergeJoin Inner (31) - : : : : :- * Sort (25) - : : : : : +- Exchange (24) - : : : : : +- * Project (23) - : : : : : +- * BroadcastHashJoin Inner BuildRight (22) - : : : : : :- * Project (17) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (16) - : : : : : : :- * Project (10) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : : : : :- * Filter (3) - : : : : : : : : +- * ColumnarToRow (2) - : : : : : : : : +- Scan parquet default.catalog_sales (1) - : : : : : : : +- BroadcastExchange (8) - : : : : : : : +- * Project (7) - : : : : : : : +- * Filter (6) - : : : : : : : +- * ColumnarToRow (5) - : : : : : : : +- Scan parquet default.household_demographics (4) - : : : : : : +- BroadcastExchange (15) - : : : : : : +- * Project (14) - : : : : : : +- * Filter (13) - : : : : : : +- * ColumnarToRow (12) - : : : : : : +- Scan parquet default.customer_demographics (11) - : : : : : +- BroadcastExchange (21) - : : : : : +- * Filter (20) - : : : : : +- * ColumnarToRow (19) - : : : : : +- Scan parquet default.date_dim (18) - : : : : +- * Sort (30) - : : : : +- Exchange (29) - : : : : +- * Filter (28) - : : : : +- * ColumnarToRow (27) - : : : : +- Scan parquet default.item (26) - : : : +- BroadcastExchange (43) - : : : +- * Project (42) - : : : +- * BroadcastHashJoin Inner BuildLeft (41) - : : : :- BroadcastExchange (37) - : : : : +- * Project (36) - : : : : +- * Filter (35) - : : : : +- * ColumnarToRow (34) - : : : : +- Scan parquet default.date_dim (33) - : : : +- * Filter (40) - : : : +- * ColumnarToRow (39) - : : : +- Scan parquet default.date_dim (38) + : : :- * Sort (34) + : : : +- Exchange (33) + : : : +- * Project (32) + : : : +- * SortMergeJoin Inner (31) + : : : :- * Sort (25) + : : : : +- Exchange (24) + : : : : +- * Project (23) + : : : : +- * BroadcastHashJoin Inner BuildRight (22) + : : : : :- * Project (17) + : : : : : +- * BroadcastHashJoin Inner BuildRight (16) + : : : : : :- * Project (10) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : : : : :- * Filter (3) + : : : : : : : +- * ColumnarToRow (2) + : : : : : : : +- Scan parquet default.catalog_sales (1) + : : : : : : +- BroadcastExchange (8) + : : : : : : +- * Project (7) + : : : : : : +- * Filter (6) + : : : : : : +- * ColumnarToRow (5) + : : : : : : +- Scan parquet default.household_demographics (4) + : : : : : +- BroadcastExchange (15) + : : : : : +- * Project (14) + : : : : : +- * Filter (13) + : : : : : +- * ColumnarToRow (12) + : : : : : +- Scan parquet default.customer_demographics (11) + : : : : +- BroadcastExchange (21) + : : : : +- * Filter (20) + : : : : +- * ColumnarToRow (19) + : : : : +- Scan parquet default.date_dim (18) + : : : +- * Sort (30) + : : : +- Exchange (29) + : : : +- * Filter (28) + : : : +- * ColumnarToRow (27) + : : : +- Scan parquet default.item (26) : : +- * Sort (58) : : +- Exchange (57) : : +- * Project (56) : : +- * BroadcastHashJoin Inner BuildRight (55) - : : :- * Filter (50) - : : : +- * ColumnarToRow (49) - : : : +- Scan parquet default.inventory (48) + : : :- * Project (50) + : : : +- * BroadcastHashJoin Inner BuildLeft (49) + : : : :- BroadcastExchange (45) + : : : : +- * Project (44) + : : : : +- * BroadcastHashJoin Inner BuildLeft (43) + : : : : :- BroadcastExchange (39) + : : : : : +- * Project (38) + : : : : : +- * Filter (37) + : : : : : +- * ColumnarToRow (36) + : : : : : +- Scan parquet default.date_dim (35) + : : : : +- * Filter (42) + : : : : +- * ColumnarToRow (41) + : : : : +- Scan parquet default.date_dim (40) + : : : +- * Filter (48) + : : : +- * ColumnarToRow (47) + : : : +- Scan parquet default.inventory (46) : : +- BroadcastExchange (54) : : +- * Filter (53) : : +- * ColumnarToRow (52) @@ -185,7 +185,7 @@ Input [8]: [cs_sold_date_sk#1, cs_ship_date_sk#2, cs_item_sk#5, cs_promo_sk#6, c (24) Exchange Input [6]: [cs_sold_date_sk#1, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, d_date#16] -Arguments: hashpartitioning(cs_item_sk#5, 5), true, [id=#18] +Arguments: hashpartitioning(cs_item_sk#5, 5), ENSURE_REQUIREMENTS, [id=#18] (25) Sort [codegen id : 5] Input [6]: [cs_sold_date_sk#1, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, d_date#16] @@ -207,101 +207,101 @@ Condition : isnotnull(i_item_sk#19) (29) Exchange Input [2]: [i_item_sk#19, i_item_desc#20] -Arguments: hashpartitioning(i_item_sk#19, 5), true, [id=#21] +Arguments: hashpartitioning(i_item_sk#19, 5), ENSURE_REQUIREMENTS, [id=#21] (30) Sort [codegen id : 7] Input [2]: [i_item_sk#19, i_item_desc#20] Arguments: [i_item_sk#19 ASC NULLS FIRST], false, 0 -(31) SortMergeJoin [codegen id : 10] +(31) SortMergeJoin [codegen id : 8] Left keys [1]: [cs_item_sk#5] Right keys [1]: [i_item_sk#19] Join condition: None -(32) Project [codegen id : 10] +(32) Project [codegen id : 8] Output [7]: [cs_sold_date_sk#1, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, d_date#16, i_item_desc#20] Input [8]: [cs_sold_date_sk#1, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, d_date#16, i_item_sk#19, i_item_desc#20] -(33) Scan parquet default.date_dim -Output [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] +(33) Exchange +Input [7]: [cs_sold_date_sk#1, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, d_date#16, i_item_desc#20] +Arguments: hashpartitioning(cs_item_sk#5, cs_sold_date_sk#1, 5), ENSURE_REQUIREMENTS, [id=#22] + +(34) Sort [codegen id : 9] +Input [7]: [cs_sold_date_sk#1, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, d_date#16, i_item_desc#20] +Arguments: [cs_item_sk#5 ASC NULLS FIRST, cs_sold_date_sk#1 ASC NULLS FIRST], false, 0 + +(35) Scan parquet default.date_dim +Output [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_year#26] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk), IsNotNull(d_week_seq), IsNotNull(d_date)] ReadSchema: struct -(34) ColumnarToRow [codegen id : 8] -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] +(36) ColumnarToRow [codegen id : 10] +Input [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_year#26] -(35) Filter [codegen id : 8] -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -Condition : ((((isnotnull(d_year#25) AND (d_year#25 = 1999)) AND isnotnull(d_date_sk#22)) AND isnotnull(d_week_seq#24)) AND isnotnull(d_date#23)) +(37) Filter [codegen id : 10] +Input [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_year#26] +Condition : ((((isnotnull(d_year#26) AND (d_year#26 = 1999)) AND isnotnull(d_date_sk#23)) AND isnotnull(d_week_seq#25)) AND isnotnull(d_date#24)) -(36) Project [codegen id : 8] -Output [3]: [d_date_sk#22, d_date#23, d_week_seq#24] -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] +(38) Project [codegen id : 10] +Output [3]: [d_date_sk#23, d_date#24, d_week_seq#25] +Input [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_year#26] -(37) BroadcastExchange -Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[2, int, true] as bigint)),false), [id=#26] +(39) BroadcastExchange +Input [3]: [d_date_sk#23, d_date#24, d_week_seq#25] +Arguments: HashedRelationBroadcastMode(List(cast(input[2, int, true] as bigint)),false), [id=#27] -(38) Scan parquet default.date_dim -Output [2]: [d_date_sk#27, d_week_seq#28] +(40) Scan parquet default.date_dim +Output [2]: [d_date_sk#28, d_week_seq#29] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(39) ColumnarToRow -Input [2]: [d_date_sk#27, d_week_seq#28] +(41) ColumnarToRow +Input [2]: [d_date_sk#28, d_week_seq#29] -(40) Filter -Input [2]: [d_date_sk#27, d_week_seq#28] -Condition : (isnotnull(d_week_seq#28) AND isnotnull(d_date_sk#27)) +(42) Filter +Input [2]: [d_date_sk#28, d_week_seq#29] +Condition : (isnotnull(d_week_seq#29) AND isnotnull(d_date_sk#28)) -(41) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [d_week_seq#24] -Right keys [1]: [d_week_seq#28] +(43) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [d_week_seq#25] +Right keys [1]: [d_week_seq#29] Join condition: None -(42) Project [codegen id : 9] -Output [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_date_sk#27] -Input [5]: [d_date_sk#22, d_date#23, d_week_seq#24, d_date_sk#27, d_week_seq#28] - -(43) BroadcastExchange -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_date_sk#27] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#29] - -(44) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#1] -Right keys [1]: [d_date_sk#22] -Join condition: (d_date#16 > d_date#23 + 5 days) - -(45) Project [codegen id : 10] -Output [7]: [cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, i_item_desc#20, d_week_seq#24, d_date_sk#27] -Input [11]: [cs_sold_date_sk#1, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, d_date#16, i_item_desc#20, d_date_sk#22, d_date#23, d_week_seq#24, d_date_sk#27] +(44) Project [codegen id : 11] +Output [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_date_sk#28] +Input [5]: [d_date_sk#23, d_date#24, d_week_seq#25, d_date_sk#28, d_week_seq#29] -(46) Exchange -Input [7]: [cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, i_item_desc#20, d_week_seq#24, d_date_sk#27] -Arguments: hashpartitioning(cs_item_sk#5, d_date_sk#27, 5), true, [id=#30] +(45) BroadcastExchange +Input [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_date_sk#28] +Arguments: HashedRelationBroadcastMode(List(cast(input[3, int, true] as bigint)),false), [id=#30] -(47) Sort [codegen id : 11] -Input [7]: [cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, i_item_desc#20, d_week_seq#24, d_date_sk#27] -Arguments: [cs_item_sk#5 ASC NULLS FIRST, d_date_sk#27 ASC NULLS FIRST], false, 0 - -(48) Scan parquet default.inventory +(46) Scan parquet default.inventory Output [4]: [inv_date_sk#31, inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34] Batched: true Location [not included in comparison]/{warehouse_dir}/inventory] PushedFilters: [IsNotNull(inv_quantity_on_hand), IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk), IsNotNull(inv_date_sk)] ReadSchema: struct -(49) ColumnarToRow [codegen id : 13] +(47) ColumnarToRow Input [4]: [inv_date_sk#31, inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34] -(50) Filter [codegen id : 13] +(48) Filter Input [4]: [inv_date_sk#31, inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34] Condition : (((isnotnull(inv_quantity_on_hand#34) AND isnotnull(inv_item_sk#32)) AND isnotnull(inv_warehouse_sk#33)) AND isnotnull(inv_date_sk#31)) +(49) BroadcastHashJoin [codegen id : 13] +Left keys [1]: [d_date_sk#28] +Right keys [1]: [inv_date_sk#31] +Join condition: None + +(50) Project [codegen id : 13] +Output [6]: [d_date_sk#23, d_date#24, d_week_seq#25, inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34] +Input [8]: [d_date_sk#23, d_date#24, d_week_seq#25, d_date_sk#28, inv_date_sk#31, inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34] + (51) Scan parquet default.warehouse Output [2]: [w_warehouse_sk#35, w_warehouse_name#36] Batched: true @@ -326,25 +326,25 @@ Right keys [1]: [w_warehouse_sk#35] Join condition: None (56) Project [codegen id : 13] -Output [4]: [inv_date_sk#31, inv_item_sk#32, inv_quantity_on_hand#34, w_warehouse_name#36] -Input [6]: [inv_date_sk#31, inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34, w_warehouse_sk#35, w_warehouse_name#36] +Output [6]: [d_date_sk#23, d_date#24, d_week_seq#25, inv_item_sk#32, inv_quantity_on_hand#34, w_warehouse_name#36] +Input [8]: [d_date_sk#23, d_date#24, d_week_seq#25, inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34, w_warehouse_sk#35, w_warehouse_name#36] (57) Exchange -Input [4]: [inv_date_sk#31, inv_item_sk#32, inv_quantity_on_hand#34, w_warehouse_name#36] -Arguments: hashpartitioning(inv_item_sk#32, inv_date_sk#31, 5), true, [id=#38] +Input [6]: [d_date_sk#23, d_date#24, d_week_seq#25, inv_item_sk#32, inv_quantity_on_hand#34, w_warehouse_name#36] +Arguments: hashpartitioning(inv_item_sk#32, d_date_sk#23, 5), ENSURE_REQUIREMENTS, [id=#38] (58) Sort [codegen id : 14] -Input [4]: [inv_date_sk#31, inv_item_sk#32, inv_quantity_on_hand#34, w_warehouse_name#36] -Arguments: [inv_item_sk#32 ASC NULLS FIRST, inv_date_sk#31 ASC NULLS FIRST], false, 0 +Input [6]: [d_date_sk#23, d_date#24, d_week_seq#25, inv_item_sk#32, inv_quantity_on_hand#34, w_warehouse_name#36] +Arguments: [inv_item_sk#32 ASC NULLS FIRST, d_date_sk#23 ASC NULLS FIRST], false, 0 (59) SortMergeJoin [codegen id : 16] -Left keys [2]: [cs_item_sk#5, d_date_sk#27] -Right keys [2]: [inv_item_sk#32, inv_date_sk#31] -Join condition: (inv_quantity_on_hand#34 < cs_quantity#8) +Left keys [2]: [cs_item_sk#5, cs_sold_date_sk#1] +Right keys [2]: [inv_item_sk#32, d_date_sk#23] +Join condition: ((inv_quantity_on_hand#34 < cs_quantity#8) AND (d_date#16 > d_date#24 + 5 days)) (60) Project [codegen id : 16] -Output [6]: [cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#24] -Input [11]: [cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, i_item_desc#20, d_week_seq#24, d_date_sk#27, inv_date_sk#31, inv_item_sk#32, inv_quantity_on_hand#34, w_warehouse_name#36] +Output [6]: [cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#25] +Input [13]: [cs_sold_date_sk#1, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, d_date#16, i_item_desc#20, d_date_sk#23, d_date#24, d_week_seq#25, inv_item_sk#32, inv_quantity_on_hand#34, w_warehouse_name#36] (61) Scan parquet default.promotion Output [1]: [p_promo_sk#39] @@ -370,15 +370,15 @@ Right keys [1]: [p_promo_sk#39] Join condition: None (66) Project [codegen id : 16] -Output [5]: [cs_item_sk#5, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#24] -Input [7]: [cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#24, p_promo_sk#39] +Output [5]: [cs_item_sk#5, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#25] +Input [7]: [cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#25, p_promo_sk#39] (67) Exchange -Input [5]: [cs_item_sk#5, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#24] -Arguments: hashpartitioning(cs_item_sk#5, cs_order_number#7, 5), true, [id=#41] +Input [5]: [cs_item_sk#5, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#25] +Arguments: hashpartitioning(cs_item_sk#5, cs_order_number#7, 5), ENSURE_REQUIREMENTS, [id=#41] (68) Sort [codegen id : 17] -Input [5]: [cs_item_sk#5, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#24] +Input [5]: [cs_item_sk#5, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#25] Arguments: [cs_item_sk#5 ASC NULLS FIRST, cs_order_number#7 ASC NULLS FIRST], false, 0 (69) Scan parquet default.catalog_returns @@ -397,7 +397,7 @@ Condition : (isnotnull(cr_item_sk#42) AND isnotnull(cr_order_number#43)) (72) Exchange Input [2]: [cr_item_sk#42, cr_order_number#43] -Arguments: hashpartitioning(cr_item_sk#42, cr_order_number#43, 5), true, [id=#44] +Arguments: hashpartitioning(cr_item_sk#42, cr_order_number#43, 5), ENSURE_REQUIREMENTS, [id=#44] (73) Sort [codegen id : 19] Input [2]: [cr_item_sk#42, cr_order_number#43] @@ -409,28 +409,28 @@ Right keys [2]: [cr_item_sk#42, cr_order_number#43] Join condition: None (75) Project [codegen id : 20] -Output [3]: [w_warehouse_name#36, i_item_desc#20, d_week_seq#24] -Input [7]: [cs_item_sk#5, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#24, cr_item_sk#42, cr_order_number#43] +Output [3]: [w_warehouse_name#36, i_item_desc#20, d_week_seq#25] +Input [7]: [cs_item_sk#5, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#25, cr_item_sk#42, cr_order_number#43] (76) HashAggregate [codegen id : 20] -Input [3]: [w_warehouse_name#36, i_item_desc#20, d_week_seq#24] -Keys [3]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#24] +Input [3]: [w_warehouse_name#36, i_item_desc#20, d_week_seq#25] +Keys [3]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#25] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#45] -Results [4]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#24, count#46] +Results [4]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#25, count#46] (77) Exchange -Input [4]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#24, count#46] -Arguments: hashpartitioning(i_item_desc#20, w_warehouse_name#36, d_week_seq#24, 5), true, [id=#47] +Input [4]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#25, count#46] +Arguments: hashpartitioning(i_item_desc#20, w_warehouse_name#36, d_week_seq#25, 5), ENSURE_REQUIREMENTS, [id=#47] (78) HashAggregate [codegen id : 21] -Input [4]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#24, count#46] -Keys [3]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#24] +Input [4]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#25, count#46] +Keys [3]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#25] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#48] -Results [6]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#24, count(1)#48 AS no_promo#49, count(1)#48 AS promo#50, count(1)#48 AS total_cnt#51] +Results [6]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#25, count(1)#48 AS no_promo#49, count(1)#48 AS promo#50, count(1)#48 AS total_cnt#51] (79) TakeOrderedAndProject -Input [6]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#24, no_promo#49, promo#50, total_cnt#51] -Arguments: 100, [total_cnt#51 DESC NULLS LAST, i_item_desc#20 ASC NULLS FIRST, w_warehouse_name#36 ASC NULLS FIRST, d_week_seq#24 ASC NULLS FIRST], [i_item_desc#20, w_warehouse_name#36, d_week_seq#24, no_promo#49, promo#50, total_cnt#51] +Input [6]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#25, no_promo#49, promo#50, total_cnt#51] +Arguments: 100, [total_cnt#51 DESC NULLS LAST, i_item_desc#20 ASC NULLS FIRST, w_warehouse_name#36 ASC NULLS FIRST, d_week_seq#25 ASC NULLS FIRST], [i_item_desc#20, w_warehouse_name#36, d_week_seq#25, no_promo#49, promo#50, total_cnt#51] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/simplified.txt index 918508787c4b0..b88505ad7b9bc 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/simplified.txt @@ -16,95 +16,95 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom Project [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] BroadcastHashJoin [cs_promo_sk,p_promo_sk] Project [cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - SortMergeJoin [cs_item_sk,d_date_sk,inv_item_sk,inv_date_sk,inv_quantity_on_hand,cs_quantity] + SortMergeJoin [cs_item_sk,cs_sold_date_sk,inv_item_sk,d_date_sk,inv_quantity_on_hand,cs_quantity,d_date,d_date] InputAdapter - WholeStageCodegen (11) - Sort [cs_item_sk,d_date_sk] + WholeStageCodegen (9) + Sort [cs_item_sk,cs_sold_date_sk] InputAdapter - Exchange [cs_item_sk,d_date_sk] #3 - WholeStageCodegen (10) - Project [cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,i_item_desc,d_week_seq,d_date_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk,d_date,d_date] - Project [cs_sold_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,d_date,i_item_desc] - SortMergeJoin [cs_item_sk,i_item_sk] - InputAdapter - WholeStageCodegen (5) - Sort [cs_item_sk] - InputAdapter - Exchange [cs_item_sk] #4 - WholeStageCodegen (4) - Project [cs_sold_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,d_date] - BroadcastHashJoin [cs_ship_date_sk,d_date_sk] - Project [cs_sold_date_sk,cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity] - BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - Project [cs_sold_date_sk,cs_ship_date_sk,cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity] - BroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk] - Filter [cs_quantity,cs_item_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_sold_date_sk,cs_ship_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity] + Exchange [cs_item_sk,cs_sold_date_sk] #3 + WholeStageCodegen (8) + Project [cs_sold_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,d_date,i_item_desc] + SortMergeJoin [cs_item_sk,i_item_sk] + InputAdapter + WholeStageCodegen (5) + Sort [cs_item_sk] + InputAdapter + Exchange [cs_item_sk] #4 + WholeStageCodegen (4) + Project [cs_sold_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,d_date] + BroadcastHashJoin [cs_ship_date_sk,d_date_sk] + Project [cs_sold_date_sk,cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity] + BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + Project [cs_sold_date_sk,cs_ship_date_sk,cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity] + BroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk] + Filter [cs_quantity,cs_item_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_sold_date_sk,cs_ship_date_sk] + ColumnarToRow InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - Project [hd_demo_sk] - Filter [hd_buy_potential,hd_demo_sk] - ColumnarToRow - InputAdapter - Scan parquet default.household_demographics [hd_demo_sk,hd_buy_potential] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (2) - Project [cd_demo_sk] - Filter [cd_marital_status,cd_demo_sk] + BroadcastExchange #5 + WholeStageCodegen (1) + Project [hd_demo_sk] + Filter [hd_buy_potential,hd_demo_sk] ColumnarToRow InputAdapter - Scan parquet default.customer_demographics [cd_demo_sk,cd_marital_status] + Scan parquet default.household_demographics [hd_demo_sk,hd_buy_potential] InputAdapter - BroadcastExchange #7 - WholeStageCodegen (3) - Filter [d_date,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date] - InputAdapter - WholeStageCodegen (7) - Sort [i_item_sk] - InputAdapter - Exchange [i_item_sk] #8 - WholeStageCodegen (6) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_item_desc] + BroadcastExchange #6 + WholeStageCodegen (2) + Project [cd_demo_sk] + Filter [cd_marital_status,cd_demo_sk] + ColumnarToRow + InputAdapter + Scan parquet default.customer_demographics [cd_demo_sk,cd_marital_status] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (3) + Filter [d_date,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_date] InputAdapter - BroadcastExchange #9 - WholeStageCodegen (9) - Project [d_date_sk,d_date,d_week_seq,d_date_sk] - BroadcastHashJoin [d_week_seq,d_week_seq] - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (8) - Project [d_date_sk,d_date,d_week_seq] - Filter [d_year,d_date_sk,d_week_seq,d_date] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - Filter [d_week_seq,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_week_seq] + WholeStageCodegen (7) + Sort [i_item_sk] + InputAdapter + Exchange [i_item_sk] #8 + WholeStageCodegen (6) + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_item_desc] InputAdapter WholeStageCodegen (14) - Sort [inv_item_sk,inv_date_sk] + Sort [inv_item_sk,d_date_sk] InputAdapter - Exchange [inv_item_sk,inv_date_sk] #11 + Exchange [inv_item_sk,d_date_sk] #9 WholeStageCodegen (13) - Project [inv_date_sk,inv_item_sk,inv_quantity_on_hand,w_warehouse_name] + Project [d_date_sk,d_date,d_week_seq,inv_item_sk,inv_quantity_on_hand,w_warehouse_name] BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Filter [inv_quantity_on_hand,inv_item_sk,inv_warehouse_sk,inv_date_sk] - ColumnarToRow + Project [d_date_sk,d_date,d_week_seq,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] + BroadcastHashJoin [d_date_sk,inv_date_sk] InputAdapter - Scan parquet default.inventory [inv_date_sk,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] + BroadcastExchange #10 + WholeStageCodegen (11) + Project [d_date_sk,d_date,d_week_seq,d_date_sk] + BroadcastHashJoin [d_week_seq,d_week_seq] + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (10) + Project [d_date_sk,d_date,d_week_seq] + Filter [d_year,d_date_sk,d_week_seq,d_date] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_date,d_week_seq,d_year] + Filter [d_week_seq,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_week_seq] + Filter [inv_quantity_on_hand,inv_item_sk,inv_warehouse_sk,inv_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.inventory [inv_date_sk,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] InputAdapter BroadcastExchange #12 WholeStageCodegen (12) 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 6e757528a3e68..6813696266ac5 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 @@ -1,343 +1,343 @@ == Physical Plan == TakeOrderedAndProject (61) +- * Project (60) - +- * BroadcastHashJoin Inner BuildRight (59) - :- * Project (38) - : +- * SortMergeJoin Inner (37) - : :- * Sort (11) - : : +- Exchange (10) - : : +- * Project (9) - : : +- * BroadcastHashJoin Inner BuildRight (8) - : : :- * Filter (3) - : : : +- * ColumnarToRow (2) - : : : +- Scan parquet default.customer (1) - : : +- BroadcastExchange (7) - : : +- * Filter (6) - : : +- * ColumnarToRow (5) - : : +- Scan parquet default.customer_address (4) - : +- * Sort (36) - : +- Exchange (35) - : +- * Filter (34) - : +- * HashAggregate (33) - : +- Exchange (32) - : +- * HashAggregate (31) - : +- * Project (30) - : +- * SortMergeJoin Inner (29) - : :- * Sort (23) - : : +- Exchange (22) - : : +- * Project (21) - : : +- * BroadcastHashJoin Inner BuildRight (20) - : : :- * Filter (14) - : : : +- * ColumnarToRow (13) - : : : +- Scan parquet default.catalog_returns (12) - : : +- BroadcastExchange (19) - : : +- * Project (18) - : : +- * Filter (17) - : : +- * ColumnarToRow (16) - : : +- Scan parquet default.date_dim (15) - : +- * Sort (28) - : +- Exchange (27) - : +- * Filter (26) - : +- * ColumnarToRow (25) - : +- Scan parquet default.customer_address (24) - +- BroadcastExchange (58) - +- * Filter (57) - +- * HashAggregate (56) - +- Exchange (55) - +- * HashAggregate (54) - +- * HashAggregate (53) - +- Exchange (52) - +- * HashAggregate (51) - +- * Project (50) - +- * SortMergeJoin Inner (49) - :- * Sort (46) - : +- Exchange (45) - : +- * Project (44) - : +- * BroadcastHashJoin Inner BuildRight (43) - : :- * Filter (41) - : : +- * ColumnarToRow (40) - : : +- Scan parquet default.catalog_returns (39) - : +- ReusedExchange (42) - +- * Sort (48) - +- ReusedExchange (47) - - -(1) Scan parquet default.customer -Output [6]: [c_customer_sk#1, c_customer_id#2, c_current_addr_sk#3, c_salutation#4, c_first_name#5, c_last_name#6] + +- * SortMergeJoin Inner (59) + :- * Sort (47) + : +- Exchange (46) + : +- * Project (45) + : +- * BroadcastHashJoin Inner BuildRight (44) + : :- * Filter (23) + : : +- * HashAggregate (22) + : : +- Exchange (21) + : : +- * HashAggregate (20) + : : +- * Project (19) + : : +- * SortMergeJoin Inner (18) + : : :- * Sort (12) + : : : +- Exchange (11) + : : : +- * Project (10) + : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : :- * Filter (3) + : : : : +- * ColumnarToRow (2) + : : : : +- Scan parquet default.catalog_returns (1) + : : : +- BroadcastExchange (8) + : : : +- * Project (7) + : : : +- * Filter (6) + : : : +- * ColumnarToRow (5) + : : : +- Scan parquet default.date_dim (4) + : : +- * Sort (17) + : : +- Exchange (16) + : : +- * Filter (15) + : : +- * ColumnarToRow (14) + : : +- Scan parquet default.customer_address (13) + : +- BroadcastExchange (43) + : +- * Filter (42) + : +- * HashAggregate (41) + : +- Exchange (40) + : +- * HashAggregate (39) + : +- * HashAggregate (38) + : +- Exchange (37) + : +- * HashAggregate (36) + : +- * Project (35) + : +- * SortMergeJoin Inner (34) + : :- * Sort (31) + : : +- Exchange (30) + : : +- * Project (29) + : : +- * BroadcastHashJoin Inner BuildRight (28) + : : :- * Filter (26) + : : : +- * ColumnarToRow (25) + : : : +- Scan parquet default.catalog_returns (24) + : : +- ReusedExchange (27) + : +- * Sort (33) + : +- ReusedExchange (32) + +- * Sort (58) + +- Exchange (57) + +- * Project (56) + +- * BroadcastHashJoin Inner BuildRight (55) + :- * Filter (50) + : +- * ColumnarToRow (49) + : +- Scan parquet default.customer (48) + +- BroadcastExchange (54) + +- * Filter (53) + +- * ColumnarToRow (52) + +- Scan parquet default.customer_address (51) + + +(1) Scan parquet default.catalog_returns +Output [4]: [cr_returned_date_sk#1, cr_returning_customer_sk#2, cr_returning_addr_sk#3, cr_return_amt_inc_tax#4] Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/catalog_returns] +PushedFilters: [IsNotNull(cr_returned_date_sk), IsNotNull(cr_returning_addr_sk), IsNotNull(cr_returning_customer_sk)] +ReadSchema: struct (2) ColumnarToRow [codegen id : 2] -Input [6]: [c_customer_sk#1, c_customer_id#2, c_current_addr_sk#3, c_salutation#4, c_first_name#5, c_last_name#6] +Input [4]: [cr_returned_date_sk#1, cr_returning_customer_sk#2, cr_returning_addr_sk#3, cr_return_amt_inc_tax#4] (3) Filter [codegen id : 2] -Input [6]: [c_customer_sk#1, c_customer_id#2, c_current_addr_sk#3, c_salutation#4, c_first_name#5, c_last_name#6] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_current_addr_sk#3)) +Input [4]: [cr_returned_date_sk#1, cr_returning_customer_sk#2, cr_returning_addr_sk#3, cr_return_amt_inc_tax#4] +Condition : ((isnotnull(cr_returned_date_sk#1) AND isnotnull(cr_returning_addr_sk#3)) AND isnotnull(cr_returning_customer_sk#2)) -(4) Scan parquet default.customer_address -Output [12]: [ca_address_sk#7, 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] +(4) Scan parquet default.date_dim +Output [2]: [d_date_sk#5, d_year#6] Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_state), EqualTo(ca_state,GA), IsNotNull(ca_address_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] +ReadSchema: struct (5) ColumnarToRow [codegen id : 1] -Input [12]: [ca_address_sk#7, 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] +Input [2]: [d_date_sk#5, d_year#6] (6) Filter [codegen id : 1] -Input [12]: [ca_address_sk#7, 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] -Condition : ((isnotnull(ca_state#14) AND (ca_state#14 = GA)) AND isnotnull(ca_address_sk#7)) +Input [2]: [d_date_sk#5, d_year#6] +Condition : ((isnotnull(d_year#6) AND (d_year#6 = 2000)) AND isnotnull(d_date_sk#5)) + +(7) Project [codegen id : 1] +Output [1]: [d_date_sk#5] +Input [2]: [d_date_sk#5, d_year#6] -(7) BroadcastExchange -Input [12]: [ca_address_sk#7, 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] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#19] +(8) BroadcastExchange +Input [1]: [d_date_sk#5] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#7] -(8) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [c_current_addr_sk#3] -Right keys [1]: [ca_address_sk#7] +(9) BroadcastHashJoin [codegen id : 2] +Left keys [1]: [cr_returned_date_sk#1] +Right keys [1]: [d_date_sk#5] Join condition: None -(9) Project [codegen id : 2] -Output [16]: [c_customer_sk#1, 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] -Input [18]: [c_customer_sk#1, c_customer_id#2, c_current_addr_sk#3, c_salutation#4, c_first_name#5, c_last_name#6, ca_address_sk#7, 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] +(10) Project [codegen id : 2] +Output [3]: [cr_returning_customer_sk#2, cr_returning_addr_sk#3, cr_return_amt_inc_tax#4] +Input [5]: [cr_returned_date_sk#1, cr_returning_customer_sk#2, cr_returning_addr_sk#3, cr_return_amt_inc_tax#4, d_date_sk#5] -(10) Exchange -Input [16]: [c_customer_sk#1, 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] -Arguments: hashpartitioning(c_customer_sk#1, 5), true, [id=#20] +(11) Exchange +Input [3]: [cr_returning_customer_sk#2, cr_returning_addr_sk#3, cr_return_amt_inc_tax#4] +Arguments: hashpartitioning(cr_returning_addr_sk#3, 5), ENSURE_REQUIREMENTS, [id=#8] -(11) Sort [codegen id : 3] -Input [16]: [c_customer_sk#1, 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] -Arguments: [c_customer_sk#1 ASC NULLS FIRST], false, 0 +(12) Sort [codegen id : 3] +Input [3]: [cr_returning_customer_sk#2, cr_returning_addr_sk#3, cr_return_amt_inc_tax#4] +Arguments: [cr_returning_addr_sk#3 ASC NULLS FIRST], false, 0 -(12) Scan parquet default.catalog_returns -Output [4]: [cr_returned_date_sk#21, cr_returning_customer_sk#22, cr_returning_addr_sk#23, cr_return_amt_inc_tax#24] +(13) Scan parquet default.customer_address +Output [2]: [ca_address_sk#9, ca_state#10] Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_returned_date_sk), IsNotNull(cr_returning_addr_sk), IsNotNull(cr_returning_customer_sk)] -ReadSchema: struct - -(13) ColumnarToRow [codegen id : 5] -Input [4]: [cr_returned_date_sk#21, cr_returning_customer_sk#22, cr_returning_addr_sk#23, cr_return_amt_inc_tax#24] +Location [not included in comparison]/{warehouse_dir}/customer_address] +PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_state)] +ReadSchema: struct -(14) Filter [codegen id : 5] -Input [4]: [cr_returned_date_sk#21, cr_returning_customer_sk#22, cr_returning_addr_sk#23, cr_return_amt_inc_tax#24] -Condition : ((isnotnull(cr_returned_date_sk#21) AND isnotnull(cr_returning_addr_sk#23)) AND isnotnull(cr_returning_customer_sk#22)) +(14) ColumnarToRow [codegen id : 4] +Input [2]: [ca_address_sk#9, ca_state#10] -(15) Scan parquet default.date_dim -Output [2]: [d_date_sk#25, d_year#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct +(15) Filter [codegen id : 4] +Input [2]: [ca_address_sk#9, ca_state#10] +Condition : (isnotnull(ca_address_sk#9) AND isnotnull(ca_state#10)) -(16) ColumnarToRow [codegen id : 4] -Input [2]: [d_date_sk#25, d_year#26] +(16) Exchange +Input [2]: [ca_address_sk#9, ca_state#10] +Arguments: hashpartitioning(ca_address_sk#9, 5), ENSURE_REQUIREMENTS, [id=#11] -(17) Filter [codegen id : 4] -Input [2]: [d_date_sk#25, d_year#26] -Condition : ((isnotnull(d_year#26) AND (d_year#26 = 2000)) AND isnotnull(d_date_sk#25)) +(17) Sort [codegen id : 5] +Input [2]: [ca_address_sk#9, ca_state#10] +Arguments: [ca_address_sk#9 ASC NULLS FIRST], false, 0 -(18) Project [codegen id : 4] -Output [1]: [d_date_sk#25] -Input [2]: [d_date_sk#25, d_year#26] +(18) SortMergeJoin [codegen id : 6] +Left keys [1]: [cr_returning_addr_sk#3] +Right keys [1]: [ca_address_sk#9] +Join condition: None -(19) BroadcastExchange -Input [1]: [d_date_sk#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#27] +(19) Project [codegen id : 6] +Output [3]: [cr_returning_customer_sk#2, cr_return_amt_inc_tax#4, ca_state#10] +Input [5]: [cr_returning_customer_sk#2, cr_returning_addr_sk#3, cr_return_amt_inc_tax#4, ca_address_sk#9, ca_state#10] + +(20) HashAggregate [codegen id : 6] +Input [3]: [cr_returning_customer_sk#2, cr_return_amt_inc_tax#4, ca_state#10] +Keys [2]: [cr_returning_customer_sk#2, ca_state#10] +Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#4))] +Aggregate Attributes [1]: [sum#12] +Results [3]: [cr_returning_customer_sk#2, ca_state#10, sum#13] + +(21) Exchange +Input [3]: [cr_returning_customer_sk#2, ca_state#10, sum#13] +Arguments: hashpartitioning(cr_returning_customer_sk#2, ca_state#10, 5), ENSURE_REQUIREMENTS, [id=#14] + +(22) HashAggregate [codegen id : 15] +Input [3]: [cr_returning_customer_sk#2, ca_state#10, sum#13] +Keys [2]: [cr_returning_customer_sk#2, ca_state#10] +Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#4))] +Aggregate Attributes [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#4))#15] +Results [3]: [cr_returning_customer_sk#2 AS ctr_customer_sk#16, ca_state#10 AS ctr_state#17, MakeDecimal(sum(UnscaledValue(cr_return_amt_inc_tax#4))#15,17,2) AS ctr_total_return#18] + +(23) Filter [codegen id : 15] +Input [3]: [ctr_customer_sk#16, ctr_state#17, ctr_total_return#18] +Condition : isnotnull(ctr_total_return#18) + +(24) Scan parquet default.catalog_returns +Output [4]: [cr_returned_date_sk#1, cr_returning_customer_sk#2, cr_returning_addr_sk#3, cr_return_amt_inc_tax#4] +Batched: true +Location [not included in comparison]/{warehouse_dir}/catalog_returns] +PushedFilters: [IsNotNull(cr_returned_date_sk), IsNotNull(cr_returning_addr_sk)] +ReadSchema: struct -(20) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [cr_returned_date_sk#21] -Right keys [1]: [d_date_sk#25] -Join condition: None +(25) ColumnarToRow [codegen id : 8] +Input [4]: [cr_returned_date_sk#1, cr_returning_customer_sk#2, cr_returning_addr_sk#3, cr_return_amt_inc_tax#4] -(21) Project [codegen id : 5] -Output [3]: [cr_returning_customer_sk#22, cr_returning_addr_sk#23, cr_return_amt_inc_tax#24] -Input [5]: [cr_returned_date_sk#21, cr_returning_customer_sk#22, cr_returning_addr_sk#23, cr_return_amt_inc_tax#24, d_date_sk#25] +(26) Filter [codegen id : 8] +Input [4]: [cr_returned_date_sk#1, cr_returning_customer_sk#2, cr_returning_addr_sk#3, cr_return_amt_inc_tax#4] +Condition : (isnotnull(cr_returned_date_sk#1) AND isnotnull(cr_returning_addr_sk#3)) -(22) Exchange -Input [3]: [cr_returning_customer_sk#22, cr_returning_addr_sk#23, cr_return_amt_inc_tax#24] -Arguments: hashpartitioning(cr_returning_addr_sk#23, 5), true, [id=#28] +(27) ReusedExchange [Reuses operator id: 8] +Output [1]: [d_date_sk#5] -(23) Sort [codegen id : 6] -Input [3]: [cr_returning_customer_sk#22, cr_returning_addr_sk#23, cr_return_amt_inc_tax#24] -Arguments: [cr_returning_addr_sk#23 ASC NULLS FIRST], false, 0 +(28) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [cr_returned_date_sk#1] +Right keys [1]: [d_date_sk#5] +Join condition: None -(24) Scan parquet default.customer_address -Output [2]: [ca_address_sk#7, ca_state#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_state)] -ReadSchema: struct +(29) Project [codegen id : 8] +Output [3]: [cr_returning_customer_sk#2, cr_returning_addr_sk#3, cr_return_amt_inc_tax#4] +Input [5]: [cr_returned_date_sk#1, cr_returning_customer_sk#2, cr_returning_addr_sk#3, cr_return_amt_inc_tax#4, d_date_sk#5] -(25) ColumnarToRow [codegen id : 7] -Input [2]: [ca_address_sk#7, ca_state#14] +(30) Exchange +Input [3]: [cr_returning_customer_sk#2, cr_returning_addr_sk#3, cr_return_amt_inc_tax#4] +Arguments: hashpartitioning(cr_returning_addr_sk#3, 5), ENSURE_REQUIREMENTS, [id=#19] -(26) Filter [codegen id : 7] -Input [2]: [ca_address_sk#7, ca_state#14] -Condition : (isnotnull(ca_address_sk#7) AND isnotnull(ca_state#14)) +(31) Sort [codegen id : 9] +Input [3]: [cr_returning_customer_sk#2, cr_returning_addr_sk#3, cr_return_amt_inc_tax#4] +Arguments: [cr_returning_addr_sk#3 ASC NULLS FIRST], false, 0 -(27) Exchange -Input [2]: [ca_address_sk#7, ca_state#14] -Arguments: hashpartitioning(ca_address_sk#7, 5), true, [id=#29] +(32) ReusedExchange [Reuses operator id: 16] +Output [2]: [ca_address_sk#9, ca_state#10] -(28) Sort [codegen id : 8] -Input [2]: [ca_address_sk#7, ca_state#14] -Arguments: [ca_address_sk#7 ASC NULLS FIRST], false, 0 +(33) Sort [codegen id : 11] +Input [2]: [ca_address_sk#9, ca_state#10] +Arguments: [ca_address_sk#9 ASC NULLS FIRST], false, 0 -(29) SortMergeJoin [codegen id : 9] -Left keys [1]: [cr_returning_addr_sk#23] -Right keys [1]: [ca_address_sk#7] +(34) SortMergeJoin [codegen id : 12] +Left keys [1]: [cr_returning_addr_sk#3] +Right keys [1]: [ca_address_sk#9] Join condition: None -(30) Project [codegen id : 9] -Output [3]: [cr_returning_customer_sk#22, cr_return_amt_inc_tax#24, ca_state#14] -Input [5]: [cr_returning_customer_sk#22, cr_returning_addr_sk#23, cr_return_amt_inc_tax#24, ca_address_sk#7, ca_state#14] - -(31) HashAggregate [codegen id : 9] -Input [3]: [cr_returning_customer_sk#22, cr_return_amt_inc_tax#24, ca_state#14] -Keys [2]: [cr_returning_customer_sk#22, ca_state#14] -Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#24))] -Aggregate Attributes [1]: [sum#30] -Results [3]: [cr_returning_customer_sk#22, ca_state#14, sum#31] - -(32) Exchange -Input [3]: [cr_returning_customer_sk#22, ca_state#14, sum#31] -Arguments: hashpartitioning(cr_returning_customer_sk#22, ca_state#14, 5), true, [id=#32] - -(33) HashAggregate [codegen id : 10] -Input [3]: [cr_returning_customer_sk#22, ca_state#14, sum#31] -Keys [2]: [cr_returning_customer_sk#22, ca_state#14] -Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#24))] -Aggregate Attributes [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#24))#33] -Results [3]: [cr_returning_customer_sk#22 AS ctr_customer_sk#34, ca_state#14 AS ctr_state#35, MakeDecimal(sum(UnscaledValue(cr_return_amt_inc_tax#24))#33,17,2) AS ctr_total_return#36] - -(34) Filter [codegen id : 10] -Input [3]: [ctr_customer_sk#34, ctr_state#35, ctr_total_return#36] -Condition : isnotnull(ctr_total_return#36) - -(35) Exchange -Input [3]: [ctr_customer_sk#34, ctr_state#35, ctr_total_return#36] -Arguments: hashpartitioning(ctr_customer_sk#34, 5), true, [id=#37] - -(36) Sort [codegen id : 11] -Input [3]: [ctr_customer_sk#34, ctr_state#35, ctr_total_return#36] -Arguments: [ctr_customer_sk#34 ASC NULLS FIRST], false, 0 - -(37) SortMergeJoin [codegen id : 20] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ctr_customer_sk#34] -Join condition: None +(35) Project [codegen id : 12] +Output [3]: [cr_returning_customer_sk#2, cr_return_amt_inc_tax#4, ca_state#10] +Input [5]: [cr_returning_customer_sk#2, cr_returning_addr_sk#3, cr_return_amt_inc_tax#4, ca_address_sk#9, ca_state#10] + +(36) HashAggregate [codegen id : 12] +Input [3]: [cr_returning_customer_sk#2, cr_return_amt_inc_tax#4, ca_state#10] +Keys [2]: [cr_returning_customer_sk#2, ca_state#10] +Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#4))] +Aggregate Attributes [1]: [sum#20] +Results [3]: [cr_returning_customer_sk#2, ca_state#10, sum#21] + +(37) Exchange +Input [3]: [cr_returning_customer_sk#2, ca_state#10, sum#21] +Arguments: hashpartitioning(cr_returning_customer_sk#2, ca_state#10, 5), ENSURE_REQUIREMENTS, [id=#22] + +(38) HashAggregate [codegen id : 13] +Input [3]: [cr_returning_customer_sk#2, ca_state#10, sum#21] +Keys [2]: [cr_returning_customer_sk#2, ca_state#10] +Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#4))] +Aggregate Attributes [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#4))#23] +Results [2]: [ca_state#10 AS ctr_state#17, MakeDecimal(sum(UnscaledValue(cr_return_amt_inc_tax#4))#23,17,2) AS ctr_total_return#18] + +(39) HashAggregate [codegen id : 13] +Input [2]: [ctr_state#17, ctr_total_return#18] +Keys [1]: [ctr_state#17] +Functions [1]: [partial_avg(ctr_total_return#18)] +Aggregate Attributes [2]: [sum#24, count#25] +Results [3]: [ctr_state#17, sum#26, count#27] + +(40) Exchange +Input [3]: [ctr_state#17, sum#26, count#27] +Arguments: hashpartitioning(ctr_state#17, 5), ENSURE_REQUIREMENTS, [id=#28] + +(41) HashAggregate [codegen id : 14] +Input [3]: [ctr_state#17, sum#26, count#27] +Keys [1]: [ctr_state#17] +Functions [1]: [avg(ctr_total_return#18)] +Aggregate Attributes [1]: [avg(ctr_total_return#18)#29] +Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#18)#29) * 1.200000), DecimalType(24,7), true) AS (CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))#30, ctr_state#17 AS ctr_state#17#31] + +(42) Filter [codegen id : 14] +Input [2]: [(CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))#30, ctr_state#17#31] +Condition : isnotnull((CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))#30) + +(43) BroadcastExchange +Input [2]: [(CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))#30, ctr_state#17#31] +Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#32] + +(44) BroadcastHashJoin [codegen id : 15] +Left keys [1]: [ctr_state#17] +Right keys [1]: [ctr_state#17#31] +Join condition: (cast(ctr_total_return#18 as decimal(24,7)) > (CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))#30) + +(45) Project [codegen id : 15] +Output [2]: [ctr_customer_sk#16, ctr_total_return#18] +Input [5]: [ctr_customer_sk#16, ctr_state#17, ctr_total_return#18, (CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))#30, ctr_state#17#31] + +(46) Exchange +Input [2]: [ctr_customer_sk#16, ctr_total_return#18] +Arguments: hashpartitioning(ctr_customer_sk#16, 5), ENSURE_REQUIREMENTS, [id=#33] + +(47) Sort [codegen id : 16] +Input [2]: [ctr_customer_sk#16, ctr_total_return#18] +Arguments: [ctr_customer_sk#16 ASC NULLS FIRST], false, 0 + +(48) Scan parquet default.customer +Output [6]: [c_customer_sk#34, c_customer_id#35, c_current_addr_sk#36, c_salutation#37, c_first_name#38, c_last_name#39] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer] +PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] +ReadSchema: struct + +(49) ColumnarToRow [codegen id : 18] +Input [6]: [c_customer_sk#34, c_customer_id#35, c_current_addr_sk#36, c_salutation#37, c_first_name#38, c_last_name#39] -(38) Project [codegen id : 20] -Output [17]: [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_state#35, ctr_total_return#36] -Input [19]: [c_customer_sk#1, 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_customer_sk#34, ctr_state#35, ctr_total_return#36] +(50) Filter [codegen id : 18] +Input [6]: [c_customer_sk#34, c_customer_id#35, c_current_addr_sk#36, c_salutation#37, c_first_name#38, c_last_name#39] +Condition : (isnotnull(c_customer_sk#34) AND isnotnull(c_current_addr_sk#36)) -(39) Scan parquet default.catalog_returns -Output [4]: [cr_returned_date_sk#21, cr_returning_customer_sk#22, cr_returning_addr_sk#23, cr_return_amt_inc_tax#24] +(51) Scan parquet default.customer_address +Output [12]: [ca_address_sk#9, ca_street_number#40, ca_street_name#41, ca_street_type#42, ca_suite_number#43, ca_city#44, ca_county#45, ca_state#10, ca_zip#46, ca_country#47, ca_gmt_offset#48, ca_location_type#49] Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_returned_date_sk), IsNotNull(cr_returning_addr_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/customer_address] +PushedFilters: [IsNotNull(ca_state), EqualTo(ca_state,GA), IsNotNull(ca_address_sk)] +ReadSchema: struct -(40) ColumnarToRow [codegen id : 13] -Input [4]: [cr_returned_date_sk#21, cr_returning_customer_sk#22, cr_returning_addr_sk#23, cr_return_amt_inc_tax#24] +(52) ColumnarToRow [codegen id : 17] +Input [12]: [ca_address_sk#9, ca_street_number#40, ca_street_name#41, ca_street_type#42, ca_suite_number#43, ca_city#44, ca_county#45, ca_state#10, ca_zip#46, ca_country#47, ca_gmt_offset#48, ca_location_type#49] -(41) Filter [codegen id : 13] -Input [4]: [cr_returned_date_sk#21, cr_returning_customer_sk#22, cr_returning_addr_sk#23, cr_return_amt_inc_tax#24] -Condition : (isnotnull(cr_returned_date_sk#21) AND isnotnull(cr_returning_addr_sk#23)) +(53) Filter [codegen id : 17] +Input [12]: [ca_address_sk#9, ca_street_number#40, ca_street_name#41, ca_street_type#42, ca_suite_number#43, ca_city#44, ca_county#45, ca_state#10, ca_zip#46, ca_country#47, ca_gmt_offset#48, ca_location_type#49] +Condition : ((isnotnull(ca_state#10) AND (ca_state#10 = GA)) AND isnotnull(ca_address_sk#9)) -(42) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#25] +(54) BroadcastExchange +Input [12]: [ca_address_sk#9, ca_street_number#40, ca_street_name#41, ca_street_type#42, ca_suite_number#43, ca_city#44, ca_county#45, ca_state#10, ca_zip#46, ca_country#47, ca_gmt_offset#48, ca_location_type#49] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#50] -(43) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [cr_returned_date_sk#21] -Right keys [1]: [d_date_sk#25] +(55) BroadcastHashJoin [codegen id : 18] +Left keys [1]: [c_current_addr_sk#36] +Right keys [1]: [ca_address_sk#9] Join condition: None -(44) Project [codegen id : 13] -Output [3]: [cr_returning_customer_sk#22, cr_returning_addr_sk#23, cr_return_amt_inc_tax#24] -Input [5]: [cr_returned_date_sk#21, cr_returning_customer_sk#22, cr_returning_addr_sk#23, cr_return_amt_inc_tax#24, d_date_sk#25] +(56) Project [codegen id : 18] +Output [16]: [c_customer_sk#34, c_customer_id#35, c_salutation#37, c_first_name#38, c_last_name#39, ca_street_number#40, ca_street_name#41, ca_street_type#42, ca_suite_number#43, ca_city#44, ca_county#45, ca_state#10, ca_zip#46, ca_country#47, ca_gmt_offset#48, ca_location_type#49] +Input [18]: [c_customer_sk#34, c_customer_id#35, c_current_addr_sk#36, c_salutation#37, c_first_name#38, c_last_name#39, ca_address_sk#9, ca_street_number#40, ca_street_name#41, ca_street_type#42, ca_suite_number#43, ca_city#44, ca_county#45, ca_state#10, ca_zip#46, ca_country#47, ca_gmt_offset#48, ca_location_type#49] -(45) Exchange -Input [3]: [cr_returning_customer_sk#22, cr_returning_addr_sk#23, cr_return_amt_inc_tax#24] -Arguments: hashpartitioning(cr_returning_addr_sk#23, 5), true, [id=#38] +(57) Exchange +Input [16]: [c_customer_sk#34, c_customer_id#35, c_salutation#37, c_first_name#38, c_last_name#39, ca_street_number#40, ca_street_name#41, ca_street_type#42, ca_suite_number#43, ca_city#44, ca_county#45, ca_state#10, ca_zip#46, ca_country#47, ca_gmt_offset#48, ca_location_type#49] +Arguments: hashpartitioning(c_customer_sk#34, 5), ENSURE_REQUIREMENTS, [id=#51] -(46) Sort [codegen id : 14] -Input [3]: [cr_returning_customer_sk#22, cr_returning_addr_sk#23, cr_return_amt_inc_tax#24] -Arguments: [cr_returning_addr_sk#23 ASC NULLS FIRST], false, 0 +(58) Sort [codegen id : 19] +Input [16]: [c_customer_sk#34, c_customer_id#35, c_salutation#37, c_first_name#38, c_last_name#39, ca_street_number#40, ca_street_name#41, ca_street_type#42, ca_suite_number#43, ca_city#44, ca_county#45, ca_state#10, ca_zip#46, ca_country#47, ca_gmt_offset#48, ca_location_type#49] +Arguments: [c_customer_sk#34 ASC NULLS FIRST], false, 0 -(47) ReusedExchange [Reuses operator id: 27] -Output [2]: [ca_address_sk#7, ca_state#14] - -(48) Sort [codegen id : 16] -Input [2]: [ca_address_sk#7, ca_state#14] -Arguments: [ca_address_sk#7 ASC NULLS FIRST], false, 0 - -(49) SortMergeJoin [codegen id : 17] -Left keys [1]: [cr_returning_addr_sk#23] -Right keys [1]: [ca_address_sk#7] +(59) SortMergeJoin [codegen id : 20] +Left keys [1]: [ctr_customer_sk#16] +Right keys [1]: [c_customer_sk#34] Join condition: None -(50) Project [codegen id : 17] -Output [3]: [cr_returning_customer_sk#22, cr_return_amt_inc_tax#24, ca_state#14] -Input [5]: [cr_returning_customer_sk#22, cr_returning_addr_sk#23, cr_return_amt_inc_tax#24, ca_address_sk#7, ca_state#14] - -(51) HashAggregate [codegen id : 17] -Input [3]: [cr_returning_customer_sk#22, cr_return_amt_inc_tax#24, ca_state#14] -Keys [2]: [cr_returning_customer_sk#22, ca_state#14] -Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#24))] -Aggregate Attributes [1]: [sum#39] -Results [3]: [cr_returning_customer_sk#22, ca_state#14, sum#40] - -(52) Exchange -Input [3]: [cr_returning_customer_sk#22, ca_state#14, sum#40] -Arguments: hashpartitioning(cr_returning_customer_sk#22, ca_state#14, 5), true, [id=#41] - -(53) HashAggregate [codegen id : 18] -Input [3]: [cr_returning_customer_sk#22, ca_state#14, sum#40] -Keys [2]: [cr_returning_customer_sk#22, ca_state#14] -Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#24))] -Aggregate Attributes [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#24))#42] -Results [2]: [ca_state#14 AS ctr_state#35, MakeDecimal(sum(UnscaledValue(cr_return_amt_inc_tax#24))#42,17,2) AS ctr_total_return#36] - -(54) HashAggregate [codegen id : 18] -Input [2]: [ctr_state#35, ctr_total_return#36] -Keys [1]: [ctr_state#35] -Functions [1]: [partial_avg(ctr_total_return#36)] -Aggregate Attributes [2]: [sum#43, count#44] -Results [3]: [ctr_state#35, sum#45, count#46] - -(55) Exchange -Input [3]: [ctr_state#35, sum#45, count#46] -Arguments: hashpartitioning(ctr_state#35, 5), true, [id=#47] - -(56) HashAggregate [codegen id : 19] -Input [3]: [ctr_state#35, sum#45, count#46] -Keys [1]: [ctr_state#35] -Functions [1]: [avg(ctr_total_return#36)] -Aggregate Attributes [1]: [avg(ctr_total_return#36)#48] -Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#36)#48) * 1.200000), DecimalType(24,7), true) AS (CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))#49, ctr_state#35 AS ctr_state#35#50] - -(57) Filter [codegen id : 19] -Input [2]: [(CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))#49, ctr_state#35#50] -Condition : isnotnull((CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))#49) - -(58) BroadcastExchange -Input [2]: [(CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))#49, ctr_state#35#50] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#51] - -(59) BroadcastHashJoin [codegen id : 20] -Left keys [1]: [ctr_state#35] -Right keys [1]: [ctr_state#35#50] -Join condition: (cast(ctr_total_return#36 as decimal(24,7)) > (CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))#49) - (60) Project [codegen id : 20] -Output [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#36] -Input [19]: [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_state#35, ctr_total_return#36, (CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))#49, ctr_state#35#50] +Output [16]: [c_customer_id#35, c_salutation#37, c_first_name#38, c_last_name#39, ca_street_number#40, ca_street_name#41, ca_street_type#42, ca_suite_number#43, ca_city#44, ca_county#45, ca_state#10, ca_zip#46, ca_country#47, ca_gmt_offset#48, ca_location_type#49, ctr_total_return#18] +Input [18]: [ctr_customer_sk#16, ctr_total_return#18, c_customer_sk#34, c_customer_id#35, c_salutation#37, c_first_name#38, c_last_name#39, ca_street_number#40, ca_street_name#41, ca_street_type#42, ca_suite_number#43, ca_city#44, ca_county#45, ca_state#10, ca_zip#46, ca_country#47, ca_gmt_offset#48, ca_location_type#49] (61) 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#36] -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#36 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#36] +Input [16]: [c_customer_id#35, c_salutation#37, c_first_name#38, c_last_name#39, ca_street_number#40, ca_street_name#41, ca_street_type#42, ca_suite_number#43, ca_city#44, ca_county#45, ca_state#10, ca_zip#46, ca_country#47, ca_gmt_offset#48, ca_location_type#49, ctr_total_return#18] +Arguments: 100, [c_customer_id#35 ASC NULLS FIRST, c_salutation#37 ASC NULLS FIRST, c_first_name#38 ASC NULLS FIRST, c_last_name#39 ASC NULLS FIRST, ca_street_number#40 ASC NULLS FIRST, ca_street_name#41 ASC NULLS FIRST, ca_street_type#42 ASC NULLS FIRST, ca_suite_number#43 ASC NULLS FIRST, ca_city#44 ASC NULLS FIRST, ca_county#45 ASC NULLS FIRST, ca_state#10 ASC NULLS FIRST, ca_zip#46 ASC NULLS FIRST, ca_country#47 ASC NULLS FIRST, ca_gmt_offset#48 ASC NULLS FIRST, ca_location_type#49 ASC NULLS FIRST, ctr_total_return#18 ASC NULLS FIRST], [c_customer_id#35, c_salutation#37, c_first_name#38, c_last_name#39, ca_street_number#40, ca_street_name#41, ca_street_type#42, ca_suite_number#43, ca_city#44, ca_county#45, ca_state#10, ca_zip#46, ca_country#47, ca_gmt_offset#48, ca_location_type#49, ctr_total_return#18] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.sf100/simplified.txt index c603ab5194286..99677b6e39736 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.sf100/simplified.txt @@ -1,48 +1,29 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type,ctr_total_return] WholeStageCodegen (20) Project [c_customer_id,c_salutation,c_first_name,c_last_name,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type,ctr_total_return] - BroadcastHashJoin [ctr_state,ctr_state,ctr_total_return,(CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))] - Project [c_customer_id,c_salutation,c_first_name,c_last_name,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type,ctr_state,ctr_total_return] - SortMergeJoin [c_customer_sk,ctr_customer_sk] - InputAdapter - WholeStageCodegen (3) - Sort [c_customer_sk] - InputAdapter - Exchange [c_customer_sk] #1 - WholeStageCodegen (2) - Project [c_customer_sk,c_customer_id,c_salutation,c_first_name,c_last_name,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Filter [c_customer_sk,c_current_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - Filter [ca_state,ca_address_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type] - InputAdapter - WholeStageCodegen (11) - Sort [ctr_customer_sk] - InputAdapter - Exchange [ctr_customer_sk] #3 - WholeStageCodegen (10) + SortMergeJoin [ctr_customer_sk,c_customer_sk] + InputAdapter + WholeStageCodegen (16) + Sort [ctr_customer_sk] + InputAdapter + Exchange [ctr_customer_sk] #1 + WholeStageCodegen (15) + Project [ctr_customer_sk,ctr_total_return] + BroadcastHashJoin [ctr_state,ctr_state,ctr_total_return,(CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))] Filter [ctr_total_return] HashAggregate [cr_returning_customer_sk,ca_state,sum] [sum(UnscaledValue(cr_return_amt_inc_tax)),ctr_customer_sk,ctr_state,ctr_total_return,sum] InputAdapter - Exchange [cr_returning_customer_sk,ca_state] #4 - WholeStageCodegen (9) + Exchange [cr_returning_customer_sk,ca_state] #2 + WholeStageCodegen (6) HashAggregate [cr_returning_customer_sk,ca_state,cr_return_amt_inc_tax] [sum,sum] Project [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] SortMergeJoin [cr_returning_addr_sk,ca_address_sk] InputAdapter - WholeStageCodegen (6) + WholeStageCodegen (3) Sort [cr_returning_addr_sk] InputAdapter - Exchange [cr_returning_addr_sk] #5 - WholeStageCodegen (5) + Exchange [cr_returning_addr_sk] #3 + WholeStageCodegen (2) Project [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] BroadcastHashJoin [cr_returned_date_sk,d_date_sk] Filter [cr_returned_date_sk,cr_returning_addr_sk,cr_returning_customer_sk] @@ -50,55 +31,74 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_st InputAdapter Scan parquet default.catalog_returns [cr_returned_date_sk,cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) + BroadcastExchange #4 + WholeStageCodegen (1) Project [d_date_sk] Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] InputAdapter - WholeStageCodegen (8) + WholeStageCodegen (5) Sort [ca_address_sk] InputAdapter - Exchange [ca_address_sk] #7 - WholeStageCodegen (7) + Exchange [ca_address_sk] #5 + WholeStageCodegen (4) Filter [ca_address_sk,ca_state] ColumnarToRow InputAdapter Scan parquet default.customer_address [ca_address_sk,ca_state] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (14) + Filter [(CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))] + HashAggregate [ctr_state,sum,count] [avg(ctr_total_return),(CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6))),ctr_state,sum,count] + InputAdapter + Exchange [ctr_state] #7 + WholeStageCodegen (13) + HashAggregate [ctr_state,ctr_total_return] [sum,count,sum,count] + HashAggregate [cr_returning_customer_sk,ca_state,sum] [sum(UnscaledValue(cr_return_amt_inc_tax)),ctr_state,ctr_total_return,sum] + InputAdapter + Exchange [cr_returning_customer_sk,ca_state] #8 + WholeStageCodegen (12) + HashAggregate [cr_returning_customer_sk,ca_state,cr_return_amt_inc_tax] [sum,sum] + Project [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] + SortMergeJoin [cr_returning_addr_sk,ca_address_sk] + InputAdapter + WholeStageCodegen (9) + Sort [cr_returning_addr_sk] + InputAdapter + Exchange [cr_returning_addr_sk] #9 + WholeStageCodegen (8) + Project [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] + BroadcastHashJoin [cr_returned_date_sk,d_date_sk] + Filter [cr_returned_date_sk,cr_returning_addr_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_returns [cr_returned_date_sk,cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] + InputAdapter + ReusedExchange [d_date_sk] #4 + InputAdapter + WholeStageCodegen (11) + Sort [ca_address_sk] + InputAdapter + ReusedExchange [ca_address_sk,ca_state] #5 InputAdapter - BroadcastExchange #8 - WholeStageCodegen (19) - Filter [(CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))] - HashAggregate [ctr_state,sum,count] [avg(ctr_total_return),(CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6))),ctr_state,sum,count] - InputAdapter - Exchange [ctr_state] #9 - WholeStageCodegen (18) - HashAggregate [ctr_state,ctr_total_return] [sum,count,sum,count] - HashAggregate [cr_returning_customer_sk,ca_state,sum] [sum(UnscaledValue(cr_return_amt_inc_tax)),ctr_state,ctr_total_return,sum] + WholeStageCodegen (19) + Sort [c_customer_sk] + InputAdapter + Exchange [c_customer_sk] #10 + WholeStageCodegen (18) + Project [c_customer_sk,c_customer_id,c_salutation,c_first_name,c_last_name,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Filter [c_customer_sk,c_current_addr_sk] + ColumnarToRow InputAdapter - Exchange [cr_returning_customer_sk,ca_state] #10 - WholeStageCodegen (17) - HashAggregate [cr_returning_customer_sk,ca_state,cr_return_amt_inc_tax] [sum,sum] - Project [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] - SortMergeJoin [cr_returning_addr_sk,ca_address_sk] - InputAdapter - WholeStageCodegen (14) - Sort [cr_returning_addr_sk] - InputAdapter - Exchange [cr_returning_addr_sk] #11 - WholeStageCodegen (13) - Project [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] - BroadcastHashJoin [cr_returned_date_sk,d_date_sk] - Filter [cr_returned_date_sk,cr_returning_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_returns [cr_returned_date_sk,cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] - InputAdapter - ReusedExchange [d_date_sk] #6 - InputAdapter - WholeStageCodegen (16) - Sort [ca_address_sk] - InputAdapter - ReusedExchange [ca_address_sk,ca_state] #7 + Scan parquet default.customer [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (17) + Filter [ca_state,ca_address_sk] + ColumnarToRow + InputAdapter + Scan parquet default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.sf100/explain.txt index 4e85516b594f7..6bcbe470cec50 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.sf100/explain.txt @@ -8,206 +8,206 @@ +- * BroadcastHashJoin Inner BuildRight (41) :- * Project (36) : +- * BroadcastHashJoin Inner BuildRight (35) - : :- * Project (23) - : : +- * BroadcastHashJoin Inner BuildRight (22) - : : :- * Project (16) - : : : +- * BroadcastHashJoin Inner BuildRight (15) - : : : :- * Project (9) - : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : :- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet default.customer (1) - : : : : +- BroadcastExchange (7) - : : : : +- * Filter (6) - : : : : +- * ColumnarToRow (5) - : : : : +- Scan parquet default.customer_demographics (4) - : : : +- BroadcastExchange (14) - : : : +- * Project (13) - : : : +- * Filter (12) - : : : +- * ColumnarToRow (11) - : : : +- Scan parquet default.household_demographics (10) - : : +- BroadcastExchange (21) - : : +- * Project (20) - : : +- * Filter (19) - : : +- * ColumnarToRow (18) - : : +- Scan parquet default.customer_address (17) + : :- * Project (30) + : : +- * BroadcastHashJoin Inner BuildRight (29) + : : :- * Project (23) + : : : +- * BroadcastHashJoin Inner BuildRight (22) + : : : :- * Project (16) + : : : : +- * BroadcastHashJoin Inner BuildLeft (15) + : : : : :- BroadcastExchange (11) + : : : : : +- * Project (10) + : : : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : : : :- * Filter (3) + : : : : : : +- * ColumnarToRow (2) + : : : : : : +- Scan parquet default.catalog_returns (1) + : : : : : +- BroadcastExchange (8) + : : : : : +- * Project (7) + : : : : : +- * Filter (6) + : : : : : +- * ColumnarToRow (5) + : : : : : +- Scan parquet default.date_dim (4) + : : : : +- * Filter (14) + : : : : +- * ColumnarToRow (13) + : : : : +- Scan parquet default.customer (12) + : : : +- BroadcastExchange (21) + : : : +- * Project (20) + : : : +- * Filter (19) + : : : +- * ColumnarToRow (18) + : : : +- Scan parquet default.household_demographics (17) + : : +- BroadcastExchange (28) + : : +- * Project (27) + : : +- * Filter (26) + : : +- * ColumnarToRow (25) + : : +- Scan parquet default.customer_address (24) : +- BroadcastExchange (34) - : +- * Project (33) - : +- * BroadcastHashJoin Inner BuildRight (32) - : :- * Filter (26) - : : +- * ColumnarToRow (25) - : : +- Scan parquet default.catalog_returns (24) - : +- BroadcastExchange (31) - : +- * Project (30) - : +- * Filter (29) - : +- * ColumnarToRow (28) - : +- Scan parquet default.date_dim (27) + : +- * Filter (33) + : +- * ColumnarToRow (32) + : +- Scan parquet default.customer_demographics (31) +- BroadcastExchange (40) +- * Filter (39) +- * ColumnarToRow (38) +- Scan parquet default.call_center (37) -(1) Scan parquet default.customer -Output [4]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_current_addr_sk#4] +(1) Scan parquet default.catalog_returns +Output [4]: [cr_returned_date_sk#1, cr_returning_customer_sk#2, cr_call_center_sk#3, cr_net_loss#4] Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_hdemo_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/catalog_returns] +PushedFilters: [IsNotNull(cr_call_center_sk), IsNotNull(cr_returned_date_sk), IsNotNull(cr_returning_customer_sk)] +ReadSchema: struct -(2) ColumnarToRow [codegen id : 7] -Input [4]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_current_addr_sk#4] +(2) ColumnarToRow [codegen id : 2] +Input [4]: [cr_returned_date_sk#1, cr_returning_customer_sk#2, cr_call_center_sk#3, cr_net_loss#4] -(3) Filter [codegen id : 7] -Input [4]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_current_addr_sk#4] -Condition : (((isnotnull(c_customer_sk#1) AND isnotnull(c_current_addr_sk#4)) AND isnotnull(c_current_cdemo_sk#2)) AND isnotnull(c_current_hdemo_sk#3)) +(3) Filter [codegen id : 2] +Input [4]: [cr_returned_date_sk#1, cr_returning_customer_sk#2, cr_call_center_sk#3, cr_net_loss#4] +Condition : ((isnotnull(cr_call_center_sk#3) AND isnotnull(cr_returned_date_sk#1)) AND isnotnull(cr_returning_customer_sk#2)) -(4) Scan parquet default.customer_demographics -Output [3]: [cd_demo_sk#5, cd_marital_status#6, cd_education_status#7] +(4) Scan parquet default.date_dim +Output [3]: [d_date_sk#5, d_year#6, d_moy#7] Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,Unknown)),And(EqualTo(cd_marital_status,W),EqualTo(cd_education_status,Advanced Degree))), IsNotNull(cd_demo_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,11), IsNotNull(d_date_sk)] +ReadSchema: struct (5) ColumnarToRow [codegen id : 1] -Input [3]: [cd_demo_sk#5, cd_marital_status#6, cd_education_status#7] +Input [3]: [d_date_sk#5, d_year#6, d_moy#7] (6) Filter [codegen id : 1] -Input [3]: [cd_demo_sk#5, cd_marital_status#6, cd_education_status#7] -Condition : ((((cd_marital_status#6 = M) AND (cd_education_status#7 = Unknown)) OR ((cd_marital_status#6 = W) AND (cd_education_status#7 = Advanced Degree))) AND isnotnull(cd_demo_sk#5)) +Input [3]: [d_date_sk#5, d_year#6, d_moy#7] +Condition : ((((isnotnull(d_year#6) AND isnotnull(d_moy#7)) AND (d_year#6 = 1998)) AND (d_moy#7 = 11)) AND isnotnull(d_date_sk#5)) + +(7) Project [codegen id : 1] +Output [1]: [d_date_sk#5] +Input [3]: [d_date_sk#5, d_year#6, d_moy#7] -(7) BroadcastExchange -Input [3]: [cd_demo_sk#5, cd_marital_status#6, cd_education_status#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#8] +(8) BroadcastExchange +Input [1]: [d_date_sk#5] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#8] -(8) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [c_current_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#5] +(9) BroadcastHashJoin [codegen id : 2] +Left keys [1]: [cr_returned_date_sk#1] +Right keys [1]: [d_date_sk#5] Join condition: None -(9) Project [codegen id : 7] -Output [5]: [c_customer_sk#1, c_current_hdemo_sk#3, c_current_addr_sk#4, cd_marital_status#6, cd_education_status#7] -Input [7]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_current_addr_sk#4, cd_demo_sk#5, cd_marital_status#6, cd_education_status#7] +(10) Project [codegen id : 2] +Output [3]: [cr_returning_customer_sk#2, cr_call_center_sk#3, cr_net_loss#4] +Input [5]: [cr_returned_date_sk#1, cr_returning_customer_sk#2, cr_call_center_sk#3, cr_net_loss#4, d_date_sk#5] -(10) Scan parquet default.household_demographics -Output [2]: [hd_demo_sk#9, hd_buy_potential#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_buy_potential), StringStartsWith(hd_buy_potential,Unknown), IsNotNull(hd_demo_sk)] -ReadSchema: struct +(11) BroadcastExchange +Input [3]: [cr_returning_customer_sk#2, cr_call_center_sk#3, cr_net_loss#4] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#9] -(11) ColumnarToRow [codegen id : 2] -Input [2]: [hd_demo_sk#9, hd_buy_potential#10] - -(12) Filter [codegen id : 2] -Input [2]: [hd_demo_sk#9, hd_buy_potential#10] -Condition : ((isnotnull(hd_buy_potential#10) AND StartsWith(hd_buy_potential#10, Unknown)) AND isnotnull(hd_demo_sk#9)) +(12) Scan parquet default.customer +Output [4]: [c_customer_sk#10, c_current_cdemo_sk#11, c_current_hdemo_sk#12, c_current_addr_sk#13] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer] +PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_hdemo_sk)] +ReadSchema: struct -(13) Project [codegen id : 2] -Output [1]: [hd_demo_sk#9] -Input [2]: [hd_demo_sk#9, hd_buy_potential#10] +(13) ColumnarToRow +Input [4]: [c_customer_sk#10, c_current_cdemo_sk#11, c_current_hdemo_sk#12, c_current_addr_sk#13] -(14) BroadcastExchange -Input [1]: [hd_demo_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#11] +(14) Filter +Input [4]: [c_customer_sk#10, c_current_cdemo_sk#11, c_current_hdemo_sk#12, c_current_addr_sk#13] +Condition : (((isnotnull(c_customer_sk#10) AND isnotnull(c_current_addr_sk#13)) AND isnotnull(c_current_cdemo_sk#11)) AND isnotnull(c_current_hdemo_sk#12)) (15) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [c_current_hdemo_sk#3] -Right keys [1]: [hd_demo_sk#9] +Left keys [1]: [cr_returning_customer_sk#2] +Right keys [1]: [c_customer_sk#10] Join condition: None (16) Project [codegen id : 7] -Output [4]: [c_customer_sk#1, c_current_addr_sk#4, cd_marital_status#6, cd_education_status#7] -Input [6]: [c_customer_sk#1, c_current_hdemo_sk#3, c_current_addr_sk#4, cd_marital_status#6, cd_education_status#7, hd_demo_sk#9] +Output [5]: [cr_call_center_sk#3, cr_net_loss#4, c_current_cdemo_sk#11, c_current_hdemo_sk#12, c_current_addr_sk#13] +Input [7]: [cr_returning_customer_sk#2, cr_call_center_sk#3, cr_net_loss#4, c_customer_sk#10, c_current_cdemo_sk#11, c_current_hdemo_sk#12, c_current_addr_sk#13] -(17) Scan parquet default.customer_address -Output [2]: [ca_address_sk#12, ca_gmt_offset#13] +(17) Scan parquet default.household_demographics +Output [2]: [hd_demo_sk#14, hd_buy_potential#15] Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_gmt_offset), EqualTo(ca_gmt_offset,-7.00), IsNotNull(ca_address_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/household_demographics] +PushedFilters: [IsNotNull(hd_buy_potential), StringStartsWith(hd_buy_potential,Unknown), IsNotNull(hd_demo_sk)] +ReadSchema: struct (18) ColumnarToRow [codegen id : 3] -Input [2]: [ca_address_sk#12, ca_gmt_offset#13] +Input [2]: [hd_demo_sk#14, hd_buy_potential#15] (19) Filter [codegen id : 3] -Input [2]: [ca_address_sk#12, ca_gmt_offset#13] -Condition : ((isnotnull(ca_gmt_offset#13) AND (ca_gmt_offset#13 = -7.00)) AND isnotnull(ca_address_sk#12)) +Input [2]: [hd_demo_sk#14, hd_buy_potential#15] +Condition : ((isnotnull(hd_buy_potential#15) AND StartsWith(hd_buy_potential#15, Unknown)) AND isnotnull(hd_demo_sk#14)) (20) Project [codegen id : 3] -Output [1]: [ca_address_sk#12] -Input [2]: [ca_address_sk#12, ca_gmt_offset#13] +Output [1]: [hd_demo_sk#14] +Input [2]: [hd_demo_sk#14, hd_buy_potential#15] (21) BroadcastExchange -Input [1]: [ca_address_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] +Input [1]: [hd_demo_sk#14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#16] (22) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [c_current_addr_sk#4] -Right keys [1]: [ca_address_sk#12] +Left keys [1]: [c_current_hdemo_sk#12] +Right keys [1]: [hd_demo_sk#14] Join condition: None (23) Project [codegen id : 7] -Output [3]: [c_customer_sk#1, cd_marital_status#6, cd_education_status#7] -Input [5]: [c_customer_sk#1, c_current_addr_sk#4, cd_marital_status#6, cd_education_status#7, ca_address_sk#12] +Output [4]: [cr_call_center_sk#3, cr_net_loss#4, c_current_cdemo_sk#11, c_current_addr_sk#13] +Input [6]: [cr_call_center_sk#3, cr_net_loss#4, c_current_cdemo_sk#11, c_current_hdemo_sk#12, c_current_addr_sk#13, hd_demo_sk#14] -(24) Scan parquet default.catalog_returns -Output [4]: [cr_returned_date_sk#15, cr_returning_customer_sk#16, cr_call_center_sk#17, cr_net_loss#18] +(24) Scan parquet default.customer_address +Output [2]: [ca_address_sk#17, ca_gmt_offset#18] Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_call_center_sk), IsNotNull(cr_returned_date_sk), IsNotNull(cr_returning_customer_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/customer_address] +PushedFilters: [IsNotNull(ca_gmt_offset), EqualTo(ca_gmt_offset,-7.00), IsNotNull(ca_address_sk)] +ReadSchema: struct -(25) ColumnarToRow [codegen id : 5] -Input [4]: [cr_returned_date_sk#15, cr_returning_customer_sk#16, cr_call_center_sk#17, cr_net_loss#18] +(25) ColumnarToRow [codegen id : 4] +Input [2]: [ca_address_sk#17, ca_gmt_offset#18] -(26) Filter [codegen id : 5] -Input [4]: [cr_returned_date_sk#15, cr_returning_customer_sk#16, cr_call_center_sk#17, cr_net_loss#18] -Condition : ((isnotnull(cr_call_center_sk#17) AND isnotnull(cr_returned_date_sk#15)) AND isnotnull(cr_returning_customer_sk#16)) +(26) Filter [codegen id : 4] +Input [2]: [ca_address_sk#17, ca_gmt_offset#18] +Condition : ((isnotnull(ca_gmt_offset#18) AND (ca_gmt_offset#18 = -7.00)) AND isnotnull(ca_address_sk#17)) -(27) Scan parquet default.date_dim -Output [3]: [d_date_sk#19, d_year#20, d_moy#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,11), IsNotNull(d_date_sk)] -ReadSchema: struct +(27) Project [codegen id : 4] +Output [1]: [ca_address_sk#17] +Input [2]: [ca_address_sk#17, ca_gmt_offset#18] -(28) ColumnarToRow [codegen id : 4] -Input [3]: [d_date_sk#19, d_year#20, d_moy#21] +(28) BroadcastExchange +Input [1]: [ca_address_sk#17] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#19] -(29) Filter [codegen id : 4] -Input [3]: [d_date_sk#19, d_year#20, d_moy#21] -Condition : ((((isnotnull(d_year#20) AND isnotnull(d_moy#21)) AND (d_year#20 = 1998)) AND (d_moy#21 = 11)) AND isnotnull(d_date_sk#19)) +(29) BroadcastHashJoin [codegen id : 7] +Left keys [1]: [c_current_addr_sk#13] +Right keys [1]: [ca_address_sk#17] +Join condition: None -(30) Project [codegen id : 4] -Output [1]: [d_date_sk#19] -Input [3]: [d_date_sk#19, d_year#20, d_moy#21] +(30) Project [codegen id : 7] +Output [3]: [cr_call_center_sk#3, cr_net_loss#4, c_current_cdemo_sk#11] +Input [5]: [cr_call_center_sk#3, cr_net_loss#4, c_current_cdemo_sk#11, c_current_addr_sk#13, ca_address_sk#17] -(31) BroadcastExchange -Input [1]: [d_date_sk#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#22] +(31) Scan parquet default.customer_demographics +Output [3]: [cd_demo_sk#20, cd_marital_status#21, cd_education_status#22] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer_demographics] +PushedFilters: [Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,Unknown)),And(EqualTo(cd_marital_status,W),EqualTo(cd_education_status,Advanced Degree))), IsNotNull(cd_demo_sk)] +ReadSchema: struct -(32) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [cr_returned_date_sk#15] -Right keys [1]: [d_date_sk#19] -Join condition: None +(32) ColumnarToRow [codegen id : 5] +Input [3]: [cd_demo_sk#20, cd_marital_status#21, cd_education_status#22] -(33) Project [codegen id : 5] -Output [3]: [cr_returning_customer_sk#16, cr_call_center_sk#17, cr_net_loss#18] -Input [5]: [cr_returned_date_sk#15, cr_returning_customer_sk#16, cr_call_center_sk#17, cr_net_loss#18, d_date_sk#19] +(33) Filter [codegen id : 5] +Input [3]: [cd_demo_sk#20, cd_marital_status#21, cd_education_status#22] +Condition : ((((cd_marital_status#21 = M) AND (cd_education_status#22 = Unknown)) OR ((cd_marital_status#21 = W) AND (cd_education_status#22 = Advanced Degree))) AND isnotnull(cd_demo_sk#20)) (34) BroadcastExchange -Input [3]: [cr_returning_customer_sk#16, cr_call_center_sk#17, cr_net_loss#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#23] +Input [3]: [cd_demo_sk#20, cd_marital_status#21, cd_education_status#22] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#23] (35) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [cr_returning_customer_sk#16] +Left keys [1]: [c_current_cdemo_sk#11] +Right keys [1]: [cd_demo_sk#20] Join condition: None (36) Project [codegen id : 7] -Output [4]: [cd_marital_status#6, cd_education_status#7, cr_call_center_sk#17, cr_net_loss#18] -Input [6]: [c_customer_sk#1, cd_marital_status#6, cd_education_status#7, cr_returning_customer_sk#16, cr_call_center_sk#17, cr_net_loss#18] +Output [4]: [cr_call_center_sk#3, cr_net_loss#4, cd_marital_status#21, cd_education_status#22] +Input [6]: [cr_call_center_sk#3, cr_net_loss#4, c_current_cdemo_sk#11, cd_demo_sk#20, cd_marital_status#21, cd_education_status#22] (37) Scan parquet default.call_center Output [4]: [cc_call_center_sk#24, cc_call_center_id#25, cc_name#26, cc_manager#27] @@ -228,35 +228,35 @@ Input [4]: [cc_call_center_sk#24, cc_call_center_id#25, cc_name#26, cc_manager#2 Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#28] (41) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [cr_call_center_sk#17] +Left keys [1]: [cr_call_center_sk#3] Right keys [1]: [cc_call_center_sk#24] Join condition: None (42) Project [codegen id : 7] -Output [6]: [cc_call_center_id#25, cc_name#26, cc_manager#27, cr_net_loss#18, cd_marital_status#6, cd_education_status#7] -Input [8]: [cd_marital_status#6, cd_education_status#7, cr_call_center_sk#17, cr_net_loss#18, cc_call_center_sk#24, cc_call_center_id#25, cc_name#26, cc_manager#27] +Output [6]: [cc_call_center_id#25, cc_name#26, cc_manager#27, cr_net_loss#4, cd_marital_status#21, cd_education_status#22] +Input [8]: [cr_call_center_sk#3, cr_net_loss#4, cd_marital_status#21, cd_education_status#22, cc_call_center_sk#24, cc_call_center_id#25, cc_name#26, cc_manager#27] (43) HashAggregate [codegen id : 7] -Input [6]: [cc_call_center_id#25, cc_name#26, cc_manager#27, cr_net_loss#18, cd_marital_status#6, cd_education_status#7] -Keys [5]: [cc_call_center_id#25, cc_name#26, cc_manager#27, cd_marital_status#6, cd_education_status#7] -Functions [1]: [partial_sum(UnscaledValue(cr_net_loss#18))] +Input [6]: [cc_call_center_id#25, cc_name#26, cc_manager#27, cr_net_loss#4, cd_marital_status#21, cd_education_status#22] +Keys [5]: [cc_call_center_id#25, cc_name#26, cc_manager#27, cd_marital_status#21, cd_education_status#22] +Functions [1]: [partial_sum(UnscaledValue(cr_net_loss#4))] Aggregate Attributes [1]: [sum#29] -Results [6]: [cc_call_center_id#25, cc_name#26, cc_manager#27, cd_marital_status#6, cd_education_status#7, sum#30] +Results [6]: [cc_call_center_id#25, cc_name#26, cc_manager#27, cd_marital_status#21, cd_education_status#22, sum#30] (44) Exchange -Input [6]: [cc_call_center_id#25, cc_name#26, cc_manager#27, cd_marital_status#6, cd_education_status#7, sum#30] -Arguments: hashpartitioning(cc_call_center_id#25, cc_name#26, cc_manager#27, cd_marital_status#6, cd_education_status#7, 5), true, [id=#31] +Input [6]: [cc_call_center_id#25, cc_name#26, cc_manager#27, cd_marital_status#21, cd_education_status#22, sum#30] +Arguments: hashpartitioning(cc_call_center_id#25, cc_name#26, cc_manager#27, cd_marital_status#21, cd_education_status#22, 5), ENSURE_REQUIREMENTS, [id=#31] (45) HashAggregate [codegen id : 8] -Input [6]: [cc_call_center_id#25, cc_name#26, cc_manager#27, cd_marital_status#6, cd_education_status#7, sum#30] -Keys [5]: [cc_call_center_id#25, cc_name#26, cc_manager#27, cd_marital_status#6, cd_education_status#7] -Functions [1]: [sum(UnscaledValue(cr_net_loss#18))] -Aggregate Attributes [1]: [sum(UnscaledValue(cr_net_loss#18))#32] -Results [4]: [cc_call_center_id#25 AS Call_Center#33, cc_name#26 AS Call_Center_Name#34, cc_manager#27 AS Manager#35, MakeDecimal(sum(UnscaledValue(cr_net_loss#18))#32,17,2) AS Returns_Loss#36] +Input [6]: [cc_call_center_id#25, cc_name#26, cc_manager#27, cd_marital_status#21, cd_education_status#22, sum#30] +Keys [5]: [cc_call_center_id#25, cc_name#26, cc_manager#27, cd_marital_status#21, cd_education_status#22] +Functions [1]: [sum(UnscaledValue(cr_net_loss#4))] +Aggregate Attributes [1]: [sum(UnscaledValue(cr_net_loss#4))#32] +Results [4]: [cc_call_center_id#25 AS Call_Center#33, cc_name#26 AS Call_Center_Name#34, cc_manager#27 AS Manager#35, MakeDecimal(sum(UnscaledValue(cr_net_loss#4))#32,17,2) AS Returns_Loss#36] (46) Exchange Input [4]: [Call_Center#33, Call_Center_Name#34, Manager#35, Returns_Loss#36] -Arguments: rangepartitioning(Returns_Loss#36 DESC NULLS LAST, 5), true, [id=#37] +Arguments: rangepartitioning(Returns_Loss#36 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, [id=#37] (47) Sort [codegen id : 9] Input [4]: [Call_Center#33, Call_Center_Name#34, Manager#35, Returns_Loss#36] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.sf100/simplified.txt index 87beb3b565cc1..6c8d629feed3e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.sf100/simplified.txt @@ -10,58 +10,58 @@ WholeStageCodegen (9) HashAggregate [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status,cr_net_loss] [sum,sum] Project [cc_call_center_id,cc_name,cc_manager,cr_net_loss,cd_marital_status,cd_education_status] BroadcastHashJoin [cr_call_center_sk,cc_call_center_sk] - Project [cd_marital_status,cd_education_status,cr_call_center_sk,cr_net_loss] - BroadcastHashJoin [c_customer_sk,cr_returning_customer_sk] - Project [c_customer_sk,cd_marital_status,cd_education_status] + Project [cr_call_center_sk,cr_net_loss,cd_marital_status,cd_education_status] + BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + Project [cr_call_center_sk,cr_net_loss,c_current_cdemo_sk] BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_customer_sk,c_current_addr_sk,cd_marital_status,cd_education_status] + Project [cr_call_center_sk,cr_net_loss,c_current_cdemo_sk,c_current_addr_sk] BroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] - Project [c_customer_sk,c_current_hdemo_sk,c_current_addr_sk,cd_marital_status,cd_education_status] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + Project [cr_call_center_sk,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] + BroadcastHashJoin [cr_returning_customer_sk,c_customer_sk] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (2) + Project [cr_returning_customer_sk,cr_call_center_sk,cr_net_loss] + BroadcastHashJoin [cr_returned_date_sk,d_date_sk] + Filter [cr_call_center_sk,cr_returned_date_sk,cr_returning_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_returns [cr_returned_date_sk,cr_returning_customer_sk,cr_call_center_sk,cr_net_loss] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (1) + Project [d_date_sk] + Filter [d_year,d_moy,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] Filter [c_customer_sk,c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk] ColumnarToRow InputAdapter Scan parquet default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Filter [cd_marital_status,cd_education_status,cd_demo_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) + BroadcastExchange #5 + WholeStageCodegen (3) Project [hd_demo_sk] Filter [hd_buy_potential,hd_demo_sk] ColumnarToRow InputAdapter Scan parquet default.household_demographics [hd_demo_sk,hd_buy_potential] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) + BroadcastExchange #6 + WholeStageCodegen (4) Project [ca_address_sk] Filter [ca_gmt_offset,ca_address_sk] ColumnarToRow InputAdapter Scan parquet default.customer_address [ca_address_sk,ca_gmt_offset] InputAdapter - BroadcastExchange #6 + BroadcastExchange #7 WholeStageCodegen (5) - Project [cr_returning_customer_sk,cr_call_center_sk,cr_net_loss] - BroadcastHashJoin [cr_returned_date_sk,d_date_sk] - Filter [cr_call_center_sk,cr_returned_date_sk,cr_returning_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_returns [cr_returned_date_sk,cr_returning_customer_sk,cr_call_center_sk,cr_net_loss] + Filter [cd_marital_status,cd_education_status,cd_demo_sk] + ColumnarToRow InputAdapter - BroadcastExchange #7 - WholeStageCodegen (4) - Project [d_date_sk] - Filter [d_year,d_moy,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year,d_moy] + Scan parquet default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] InputAdapter BroadcastExchange #8 WholeStageCodegen (6) 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 2d76deefcaa36..f6c5258701525 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 @@ -34,24 +34,24 @@ TakeOrderedAndProject (160) : +- * Sort (46) : +- Exchange (45) : +- * Project (44) - : +- * SortMergeJoin Inner (43) - : :- * Sort (37) - : : +- Exchange (36) - : : +- * Project (35) - : : +- * BroadcastHashJoin Inner BuildRight (34) - : : :- * Project (29) - : : : +- * Filter (28) - : : : +- * ColumnarToRow (27) - : : : +- Scan parquet default.customer (26) - : : +- BroadcastExchange (33) - : : +- * Filter (32) - : : +- * ColumnarToRow (31) - : : +- Scan parquet default.customer_address (30) - : +- * Sort (42) - : +- Exchange (41) - : +- * Filter (40) - : +- * ColumnarToRow (39) - : +- Scan parquet default.customer_demographics (38) + : +- * BroadcastHashJoin Inner BuildRight (43) + : :- * Project (38) + : : +- * SortMergeJoin Inner (37) + : : :- * Sort (31) + : : : +- Exchange (30) + : : : +- * Project (29) + : : : +- * Filter (28) + : : : +- * ColumnarToRow (27) + : : : +- Scan parquet default.customer (26) + : : +- * Sort (36) + : : +- Exchange (35) + : : +- * Filter (34) + : : +- * ColumnarToRow (33) + : : +- Scan parquet default.customer_demographics (32) + : +- BroadcastExchange (42) + : +- * Filter (41) + : +- * ColumnarToRow (40) + : +- Scan parquet default.customer_address (39) :- * HashAggregate (76) : +- Exchange (75) : +- * HashAggregate (74) @@ -266,7 +266,7 @@ Input [10]: [cs_bill_customer_sk#2, cs_item_sk#4, cs_quantity#5, cs_list_price#6 (24) Exchange Input [8]: [cs_bill_customer_sk#2, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, i_item_id#19] -Arguments: hashpartitioning(cs_bill_customer_sk#2, 5), true, [id=#21] +Arguments: hashpartitioning(cs_bill_customer_sk#2, 5), ENSURE_REQUIREMENTS, [id=#21] (25) Sort [codegen id : 5] Input [8]: [cs_bill_customer_sk#2, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, i_item_id#19] @@ -279,89 +279,89 @@ Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [In(c_birth_month, [9,5,12,4,1,10]), IsNotNull(c_customer_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct -(27) ColumnarToRow [codegen id : 7] +(27) ColumnarToRow [codegen id : 6] Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_month#25, c_birth_year#26] -(28) Filter [codegen id : 7] +(28) Filter [codegen id : 6] Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_month#25, c_birth_year#26] Condition : (((c_birth_month#25 IN (9,5,12,4,1,10) AND isnotnull(c_customer_sk#22)) AND isnotnull(c_current_cdemo_sk#23)) AND isnotnull(c_current_addr_sk#24)) -(29) Project [codegen id : 7] +(29) Project [codegen id : 6] Output [4]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_year#26] Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_month#25, c_birth_year#26] -(30) Scan parquet default.customer_address -Output [4]: [ca_address_sk#27, ca_county#28, ca_state#29, ca_country#30] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [In(ca_state, [ND,WI,AL,NC,OK,MS,TN]), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(31) ColumnarToRow [codegen id : 6] -Input [4]: [ca_address_sk#27, ca_county#28, ca_state#29, ca_country#30] - -(32) Filter [codegen id : 6] -Input [4]: [ca_address_sk#27, ca_county#28, ca_state#29, ca_country#30] -Condition : (ca_state#29 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#27)) - -(33) BroadcastExchange -Input [4]: [ca_address_sk#27, ca_county#28, ca_state#29, ca_country#30] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#31] - -(34) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [c_current_addr_sk#24] -Right keys [1]: [ca_address_sk#27] -Join condition: None - -(35) Project [codegen id : 7] -Output [6]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26, ca_county#28, ca_state#29, ca_country#30] -Input [8]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_year#26, ca_address_sk#27, ca_county#28, ca_state#29, ca_country#30] - -(36) Exchange -Input [6]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26, ca_county#28, ca_state#29, ca_country#30] -Arguments: hashpartitioning(c_current_cdemo_sk#23, 5), true, [id=#32] +(30) Exchange +Input [4]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_year#26] +Arguments: hashpartitioning(c_current_cdemo_sk#23, 5), ENSURE_REQUIREMENTS, [id=#27] -(37) Sort [codegen id : 8] -Input [6]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26, ca_county#28, ca_state#29, ca_country#30] +(31) Sort [codegen id : 7] +Input [4]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_year#26] Arguments: [c_current_cdemo_sk#23 ASC NULLS FIRST], false, 0 -(38) Scan parquet default.customer_demographics -Output [1]: [cd_demo_sk#33] +(32) Scan parquet default.customer_demographics +Output [1]: [cd_demo_sk#28] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(39) ColumnarToRow [codegen id : 9] -Input [1]: [cd_demo_sk#33] +(33) ColumnarToRow [codegen id : 8] +Input [1]: [cd_demo_sk#28] -(40) Filter [codegen id : 9] -Input [1]: [cd_demo_sk#33] -Condition : isnotnull(cd_demo_sk#33) +(34) Filter [codegen id : 8] +Input [1]: [cd_demo_sk#28] +Condition : isnotnull(cd_demo_sk#28) -(41) Exchange -Input [1]: [cd_demo_sk#33] -Arguments: hashpartitioning(cd_demo_sk#33, 5), true, [id=#34] +(35) Exchange +Input [1]: [cd_demo_sk#28] +Arguments: hashpartitioning(cd_demo_sk#28, 5), ENSURE_REQUIREMENTS, [id=#29] -(42) Sort [codegen id : 10] -Input [1]: [cd_demo_sk#33] -Arguments: [cd_demo_sk#33 ASC NULLS FIRST], false, 0 +(36) Sort [codegen id : 9] +Input [1]: [cd_demo_sk#28] +Arguments: [cd_demo_sk#28 ASC NULLS FIRST], false, 0 -(43) SortMergeJoin [codegen id : 11] +(37) SortMergeJoin [codegen id : 11] Left keys [1]: [c_current_cdemo_sk#23] -Right keys [1]: [cd_demo_sk#33] +Right keys [1]: [cd_demo_sk#28] +Join condition: None + +(38) Project [codegen id : 11] +Output [3]: [c_customer_sk#22, c_current_addr_sk#24, c_birth_year#26] +Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_year#26, cd_demo_sk#28] + +(39) Scan parquet default.customer_address +Output [4]: [ca_address_sk#30, ca_county#31, ca_state#32, ca_country#33] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer_address] +PushedFilters: [In(ca_state, [ND,WI,AL,NC,OK,MS,TN]), IsNotNull(ca_address_sk)] +ReadSchema: struct + +(40) ColumnarToRow [codegen id : 10] +Input [4]: [ca_address_sk#30, ca_county#31, ca_state#32, ca_country#33] + +(41) Filter [codegen id : 10] +Input [4]: [ca_address_sk#30, ca_county#31, ca_state#32, ca_country#33] +Condition : (ca_state#32 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#30)) + +(42) BroadcastExchange +Input [4]: [ca_address_sk#30, ca_county#31, ca_state#32, ca_country#33] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#34] + +(43) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [c_current_addr_sk#24] +Right keys [1]: [ca_address_sk#30] Join condition: None (44) Project [codegen id : 11] -Output [5]: [c_customer_sk#22, c_birth_year#26, ca_county#28, ca_state#29, ca_country#30] -Input [7]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26, ca_county#28, ca_state#29, ca_country#30, cd_demo_sk#33] +Output [5]: [c_customer_sk#22, c_birth_year#26, ca_county#31, ca_state#32, ca_country#33] +Input [7]: [c_customer_sk#22, c_current_addr_sk#24, c_birth_year#26, ca_address_sk#30, ca_county#31, ca_state#32, ca_country#33] (45) Exchange -Input [5]: [c_customer_sk#22, c_birth_year#26, ca_county#28, ca_state#29, ca_country#30] -Arguments: hashpartitioning(c_customer_sk#22, 5), true, [id=#35] +Input [5]: [c_customer_sk#22, c_birth_year#26, ca_county#31, ca_state#32, ca_country#33] +Arguments: hashpartitioning(c_customer_sk#22, 5), ENSURE_REQUIREMENTS, [id=#35] (46) Sort [codegen id : 12] -Input [5]: [c_customer_sk#22, c_birth_year#26, ca_county#28, ca_state#29, ca_country#30] +Input [5]: [c_customer_sk#22, c_birth_year#26, ca_county#31, ca_state#32, ca_country#33] Arguments: [c_customer_sk#22 ASC NULLS FIRST], false, 0 (47) SortMergeJoin [codegen id : 13] @@ -370,26 +370,26 @@ Right keys [1]: [c_customer_sk#22] Join condition: None (48) Project [codegen id : 13] -Output [11]: [i_item_id#19, ca_country#30, ca_state#29, ca_county#28, cast(cs_quantity#5 as decimal(12,2)) AS agg1#36, cast(cs_list_price#6 as decimal(12,2)) AS agg2#37, cast(cs_coupon_amt#8 as decimal(12,2)) AS agg3#38, cast(cs_sales_price#7 as decimal(12,2)) AS agg4#39, cast(cs_net_profit#9 as decimal(12,2)) AS agg5#40, cast(c_birth_year#26 as decimal(12,2)) AS agg6#41, cast(cd_dep_count#13 as decimal(12,2)) AS agg7#42] -Input [13]: [cs_bill_customer_sk#2, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, i_item_id#19, c_customer_sk#22, c_birth_year#26, ca_county#28, ca_state#29, ca_country#30] +Output [11]: [i_item_id#19, ca_country#33, ca_state#32, ca_county#31, cast(cs_quantity#5 as decimal(12,2)) AS agg1#36, cast(cs_list_price#6 as decimal(12,2)) AS agg2#37, cast(cs_coupon_amt#8 as decimal(12,2)) AS agg3#38, cast(cs_sales_price#7 as decimal(12,2)) AS agg4#39, cast(cs_net_profit#9 as decimal(12,2)) AS agg5#40, cast(c_birth_year#26 as decimal(12,2)) AS agg6#41, cast(cd_dep_count#13 as decimal(12,2)) AS agg7#42] +Input [13]: [cs_bill_customer_sk#2, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, i_item_id#19, c_customer_sk#22, c_birth_year#26, ca_county#31, ca_state#32, ca_country#33] (49) HashAggregate [codegen id : 13] -Input [11]: [i_item_id#19, ca_country#30, ca_state#29, ca_county#28, agg1#36, agg2#37, agg3#38, agg4#39, agg5#40, agg6#41, agg7#42] -Keys [4]: [i_item_id#19, ca_country#30, ca_state#29, ca_county#28] +Input [11]: [i_item_id#19, ca_country#33, ca_state#32, ca_county#31, agg1#36, agg2#37, agg3#38, agg4#39, agg5#40, agg6#41, agg7#42] +Keys [4]: [i_item_id#19, ca_country#33, ca_state#32, ca_county#31] Functions [7]: [partial_avg(agg1#36), partial_avg(agg2#37), partial_avg(agg3#38), partial_avg(agg4#39), partial_avg(agg5#40), partial_avg(agg6#41), partial_avg(agg7#42)] Aggregate Attributes [14]: [sum#43, count#44, sum#45, count#46, sum#47, count#48, sum#49, count#50, sum#51, count#52, sum#53, count#54, sum#55, count#56] -Results [18]: [i_item_id#19, ca_country#30, ca_state#29, ca_county#28, sum#57, count#58, sum#59, count#60, sum#61, count#62, sum#63, count#64, sum#65, count#66, sum#67, count#68, sum#69, count#70] +Results [18]: [i_item_id#19, ca_country#33, ca_state#32, ca_county#31, sum#57, count#58, sum#59, count#60, sum#61, count#62, sum#63, count#64, sum#65, count#66, sum#67, count#68, sum#69, count#70] (50) Exchange -Input [18]: [i_item_id#19, ca_country#30, ca_state#29, ca_county#28, sum#57, count#58, sum#59, count#60, sum#61, count#62, sum#63, count#64, sum#65, count#66, sum#67, count#68, sum#69, count#70] -Arguments: hashpartitioning(i_item_id#19, ca_country#30, ca_state#29, ca_county#28, 5), true, [id=#71] +Input [18]: [i_item_id#19, ca_country#33, ca_state#32, ca_county#31, sum#57, count#58, sum#59, count#60, sum#61, count#62, sum#63, count#64, sum#65, count#66, sum#67, count#68, sum#69, count#70] +Arguments: hashpartitioning(i_item_id#19, ca_country#33, ca_state#32, ca_county#31, 5), ENSURE_REQUIREMENTS, [id=#71] (51) HashAggregate [codegen id : 14] -Input [18]: [i_item_id#19, ca_country#30, ca_state#29, ca_county#28, sum#57, count#58, sum#59, count#60, sum#61, count#62, sum#63, count#64, sum#65, count#66, sum#67, count#68, sum#69, count#70] -Keys [4]: [i_item_id#19, ca_country#30, ca_state#29, ca_county#28] +Input [18]: [i_item_id#19, ca_country#33, ca_state#32, ca_county#31, sum#57, count#58, sum#59, count#60, sum#61, count#62, sum#63, count#64, sum#65, count#66, sum#67, count#68, sum#69, count#70] +Keys [4]: [i_item_id#19, ca_country#33, ca_state#32, ca_county#31] Functions [7]: [avg(agg1#36), avg(agg2#37), avg(agg3#38), avg(agg4#39), avg(agg5#40), avg(agg6#41), avg(agg7#42)] Aggregate Attributes [7]: [avg(agg1#36)#72, avg(agg2#37)#73, avg(agg3#38)#74, avg(agg4#39)#75, avg(agg5#40)#76, avg(agg6#41)#77, avg(agg7#42)#78] -Results [11]: [i_item_id#19, ca_country#30, ca_state#29, ca_county#28, avg(agg1#36)#72 AS agg1#79, avg(agg2#37)#73 AS agg2#80, avg(agg3#38)#74 AS agg3#81, avg(agg4#39)#75 AS agg4#82, avg(agg5#40)#76 AS agg5#83, avg(agg6#41)#77 AS agg6#84, avg(agg7#42)#78 AS agg7#85] +Results [11]: [i_item_id#19, ca_country#33, ca_state#32, ca_county#31, avg(agg1#36)#72 AS agg1#79, avg(agg2#37)#73 AS agg2#80, avg(agg3#38)#74 AS agg3#81, avg(agg4#39)#75 AS agg4#82, avg(agg5#40)#76 AS agg5#83, avg(agg6#41)#77 AS agg6#84, avg(agg7#42)#78 AS agg7#85] (52) ReusedExchange [Reuses operator id: 24] Output [8]: [cs_bill_customer_sk#2, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, i_item_id#19] @@ -417,41 +417,41 @@ Output [4]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_bi Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_month#25, c_birth_year#26] (58) Scan parquet default.customer_address -Output [3]: [ca_address_sk#27, ca_state#29, ca_country#30] +Output [3]: [ca_address_sk#30, ca_state#32, ca_country#33] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_state, [ND,WI,AL,NC,OK,MS,TN]), IsNotNull(ca_address_sk)] ReadSchema: struct (59) ColumnarToRow [codegen id : 20] -Input [3]: [ca_address_sk#27, ca_state#29, ca_country#30] +Input [3]: [ca_address_sk#30, ca_state#32, ca_country#33] (60) Filter [codegen id : 20] -Input [3]: [ca_address_sk#27, ca_state#29, ca_country#30] -Condition : (ca_state#29 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#27)) +Input [3]: [ca_address_sk#30, ca_state#32, ca_country#33] +Condition : (ca_state#32 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#30)) (61) BroadcastExchange -Input [3]: [ca_address_sk#27, ca_state#29, ca_country#30] +Input [3]: [ca_address_sk#30, ca_state#32, ca_country#33] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#86] (62) BroadcastHashJoin [codegen id : 21] Left keys [1]: [c_current_addr_sk#24] -Right keys [1]: [ca_address_sk#27] +Right keys [1]: [ca_address_sk#30] Join condition: None (63) Project [codegen id : 21] -Output [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26, ca_state#29, ca_country#30] -Input [7]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_year#26, ca_address_sk#27, ca_state#29, ca_country#30] +Output [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26, ca_state#32, ca_country#33] +Input [7]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_year#26, ca_address_sk#30, ca_state#32, ca_country#33] (64) Exchange -Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26, ca_state#29, ca_country#30] -Arguments: hashpartitioning(c_current_cdemo_sk#23, 5), true, [id=#87] +Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26, ca_state#32, ca_country#33] +Arguments: hashpartitioning(c_current_cdemo_sk#23, 5), ENSURE_REQUIREMENTS, [id=#87] (65) Sort [codegen id : 22] -Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26, ca_state#29, ca_country#30] +Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26, ca_state#32, ca_country#33] Arguments: [c_current_cdemo_sk#23 ASC NULLS FIRST], false, 0 -(66) ReusedExchange [Reuses operator id: 41] +(66) ReusedExchange [Reuses operator id: 35] Output [1]: [cd_demo_sk#88] (67) Sort [codegen id : 24] @@ -464,15 +464,15 @@ Right keys [1]: [cd_demo_sk#88] Join condition: None (69) Project [codegen id : 25] -Output [4]: [c_customer_sk#22, c_birth_year#26, ca_state#29, ca_country#30] -Input [6]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26, ca_state#29, ca_country#30, cd_demo_sk#88] +Output [4]: [c_customer_sk#22, c_birth_year#26, ca_state#32, ca_country#33] +Input [6]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26, ca_state#32, ca_country#33, cd_demo_sk#88] (70) Exchange -Input [4]: [c_customer_sk#22, c_birth_year#26, ca_state#29, ca_country#30] -Arguments: hashpartitioning(c_customer_sk#22, 5), true, [id=#89] +Input [4]: [c_customer_sk#22, c_birth_year#26, ca_state#32, ca_country#33] +Arguments: hashpartitioning(c_customer_sk#22, 5), ENSURE_REQUIREMENTS, [id=#89] (71) Sort [codegen id : 26] -Input [4]: [c_customer_sk#22, c_birth_year#26, ca_state#29, ca_country#30] +Input [4]: [c_customer_sk#22, c_birth_year#26, ca_state#32, ca_country#33] Arguments: [c_customer_sk#22 ASC NULLS FIRST], false, 0 (72) SortMergeJoin [codegen id : 27] @@ -481,26 +481,26 @@ Right keys [1]: [c_customer_sk#22] Join condition: None (73) Project [codegen id : 27] -Output [10]: [i_item_id#19, ca_country#30, ca_state#29, cast(cs_quantity#5 as decimal(12,2)) AS agg1#36, cast(cs_list_price#6 as decimal(12,2)) AS agg2#37, cast(cs_coupon_amt#8 as decimal(12,2)) AS agg3#38, cast(cs_sales_price#7 as decimal(12,2)) AS agg4#39, cast(cs_net_profit#9 as decimal(12,2)) AS agg5#40, cast(c_birth_year#26 as decimal(12,2)) AS agg6#41, cast(cd_dep_count#13 as decimal(12,2)) AS agg7#42] -Input [12]: [cs_bill_customer_sk#2, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, i_item_id#19, c_customer_sk#22, c_birth_year#26, ca_state#29, ca_country#30] +Output [10]: [i_item_id#19, ca_country#33, ca_state#32, cast(cs_quantity#5 as decimal(12,2)) AS agg1#36, cast(cs_list_price#6 as decimal(12,2)) AS agg2#37, cast(cs_coupon_amt#8 as decimal(12,2)) AS agg3#38, cast(cs_sales_price#7 as decimal(12,2)) AS agg4#39, cast(cs_net_profit#9 as decimal(12,2)) AS agg5#40, cast(c_birth_year#26 as decimal(12,2)) AS agg6#41, cast(cd_dep_count#13 as decimal(12,2)) AS agg7#42] +Input [12]: [cs_bill_customer_sk#2, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, i_item_id#19, c_customer_sk#22, c_birth_year#26, ca_state#32, ca_country#33] (74) HashAggregate [codegen id : 27] -Input [10]: [i_item_id#19, ca_country#30, ca_state#29, agg1#36, agg2#37, agg3#38, agg4#39, agg5#40, agg6#41, agg7#42] -Keys [3]: [i_item_id#19, ca_country#30, ca_state#29] +Input [10]: [i_item_id#19, ca_country#33, ca_state#32, agg1#36, agg2#37, agg3#38, agg4#39, agg5#40, agg6#41, agg7#42] +Keys [3]: [i_item_id#19, ca_country#33, ca_state#32] Functions [7]: [partial_avg(agg1#36), partial_avg(agg2#37), partial_avg(agg3#38), partial_avg(agg4#39), partial_avg(agg5#40), partial_avg(agg6#41), partial_avg(agg7#42)] Aggregate Attributes [14]: [sum#90, count#91, sum#92, count#93, sum#94, count#95, sum#96, count#97, sum#98, count#99, sum#100, count#101, sum#102, count#103] -Results [17]: [i_item_id#19, ca_country#30, ca_state#29, sum#104, count#105, sum#106, count#107, sum#108, count#109, sum#110, count#111, sum#112, count#113, sum#114, count#115, sum#116, count#117] +Results [17]: [i_item_id#19, ca_country#33, ca_state#32, sum#104, count#105, sum#106, count#107, sum#108, count#109, sum#110, count#111, sum#112, count#113, sum#114, count#115, sum#116, count#117] (75) Exchange -Input [17]: [i_item_id#19, ca_country#30, ca_state#29, sum#104, count#105, sum#106, count#107, sum#108, count#109, sum#110, count#111, sum#112, count#113, sum#114, count#115, sum#116, count#117] -Arguments: hashpartitioning(i_item_id#19, ca_country#30, ca_state#29, 5), true, [id=#118] +Input [17]: [i_item_id#19, ca_country#33, ca_state#32, sum#104, count#105, sum#106, count#107, sum#108, count#109, sum#110, count#111, sum#112, count#113, sum#114, count#115, sum#116, count#117] +Arguments: hashpartitioning(i_item_id#19, ca_country#33, ca_state#32, 5), ENSURE_REQUIREMENTS, [id=#118] (76) HashAggregate [codegen id : 28] -Input [17]: [i_item_id#19, ca_country#30, ca_state#29, sum#104, count#105, sum#106, count#107, sum#108, count#109, sum#110, count#111, sum#112, count#113, sum#114, count#115, sum#116, count#117] -Keys [3]: [i_item_id#19, ca_country#30, ca_state#29] +Input [17]: [i_item_id#19, ca_country#33, ca_state#32, sum#104, count#105, sum#106, count#107, sum#108, count#109, sum#110, count#111, sum#112, count#113, sum#114, count#115, sum#116, count#117] +Keys [3]: [i_item_id#19, ca_country#33, ca_state#32] Functions [7]: [avg(agg1#36), avg(agg2#37), avg(agg3#38), avg(agg4#39), avg(agg5#40), avg(agg6#41), avg(agg7#42)] Aggregate Attributes [7]: [avg(agg1#36)#119, avg(agg2#37)#120, avg(agg3#38)#121, avg(agg4#39)#122, avg(agg5#40)#123, avg(agg6#41)#124, avg(agg7#42)#125] -Results [11]: [i_item_id#19, ca_country#30, ca_state#29, null AS county#126, avg(agg1#36)#119 AS agg1#127, avg(agg2#37)#120 AS agg2#128, avg(agg3#38)#121 AS agg3#129, avg(agg4#39)#122 AS agg4#130, avg(agg5#40)#123 AS agg5#131, avg(agg6#41)#124 AS agg6#132, avg(agg7#42)#125 AS agg7#133] +Results [11]: [i_item_id#19, ca_country#33, ca_state#32, null AS county#126, avg(agg1#36)#119 AS agg1#127, avg(agg2#37)#120 AS agg2#128, avg(agg3#38)#121 AS agg3#129, avg(agg4#39)#122 AS agg4#130, avg(agg5#40)#123 AS agg5#131, avg(agg6#41)#124 AS agg6#132, avg(agg7#42)#125 AS agg7#133] (77) ReusedExchange [Reuses operator id: 24] Output [8]: [cs_bill_customer_sk#2, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, i_item_id#19] @@ -528,45 +528,45 @@ Output [4]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_bi Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_month#25, c_birth_year#26] (83) Scan parquet default.customer_address -Output [3]: [ca_address_sk#27, ca_state#29, ca_country#30] +Output [3]: [ca_address_sk#30, ca_state#32, ca_country#33] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_state, [ND,WI,AL,NC,OK,MS,TN]), IsNotNull(ca_address_sk)] ReadSchema: struct (84) ColumnarToRow [codegen id : 34] -Input [3]: [ca_address_sk#27, ca_state#29, ca_country#30] +Input [3]: [ca_address_sk#30, ca_state#32, ca_country#33] (85) Filter [codegen id : 34] -Input [3]: [ca_address_sk#27, ca_state#29, ca_country#30] -Condition : (ca_state#29 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#27)) +Input [3]: [ca_address_sk#30, ca_state#32, ca_country#33] +Condition : (ca_state#32 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#30)) (86) Project [codegen id : 34] -Output [2]: [ca_address_sk#27, ca_country#30] -Input [3]: [ca_address_sk#27, ca_state#29, ca_country#30] +Output [2]: [ca_address_sk#30, ca_country#33] +Input [3]: [ca_address_sk#30, ca_state#32, ca_country#33] (87) BroadcastExchange -Input [2]: [ca_address_sk#27, ca_country#30] +Input [2]: [ca_address_sk#30, ca_country#33] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#134] (88) BroadcastHashJoin [codegen id : 35] Left keys [1]: [c_current_addr_sk#24] -Right keys [1]: [ca_address_sk#27] +Right keys [1]: [ca_address_sk#30] Join condition: None (89) Project [codegen id : 35] -Output [4]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26, ca_country#30] -Input [6]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_year#26, ca_address_sk#27, ca_country#30] +Output [4]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26, ca_country#33] +Input [6]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_year#26, ca_address_sk#30, ca_country#33] (90) Exchange -Input [4]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26, ca_country#30] -Arguments: hashpartitioning(c_current_cdemo_sk#23, 5), true, [id=#135] +Input [4]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26, ca_country#33] +Arguments: hashpartitioning(c_current_cdemo_sk#23, 5), ENSURE_REQUIREMENTS, [id=#135] (91) Sort [codegen id : 36] -Input [4]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26, ca_country#30] +Input [4]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26, ca_country#33] Arguments: [c_current_cdemo_sk#23 ASC NULLS FIRST], false, 0 -(92) ReusedExchange [Reuses operator id: 41] +(92) ReusedExchange [Reuses operator id: 35] Output [1]: [cd_demo_sk#136] (93) Sort [codegen id : 38] @@ -579,15 +579,15 @@ Right keys [1]: [cd_demo_sk#136] Join condition: None (95) Project [codegen id : 39] -Output [3]: [c_customer_sk#22, c_birth_year#26, ca_country#30] -Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26, ca_country#30, cd_demo_sk#136] +Output [3]: [c_customer_sk#22, c_birth_year#26, ca_country#33] +Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26, ca_country#33, cd_demo_sk#136] (96) Exchange -Input [3]: [c_customer_sk#22, c_birth_year#26, ca_country#30] -Arguments: hashpartitioning(c_customer_sk#22, 5), true, [id=#137] +Input [3]: [c_customer_sk#22, c_birth_year#26, ca_country#33] +Arguments: hashpartitioning(c_customer_sk#22, 5), ENSURE_REQUIREMENTS, [id=#137] (97) Sort [codegen id : 40] -Input [3]: [c_customer_sk#22, c_birth_year#26, ca_country#30] +Input [3]: [c_customer_sk#22, c_birth_year#26, ca_country#33] Arguments: [c_customer_sk#22 ASC NULLS FIRST], false, 0 (98) SortMergeJoin [codegen id : 41] @@ -596,26 +596,26 @@ Right keys [1]: [c_customer_sk#22] Join condition: None (99) Project [codegen id : 41] -Output [9]: [i_item_id#19, ca_country#30, cast(cs_quantity#5 as decimal(12,2)) AS agg1#36, cast(cs_list_price#6 as decimal(12,2)) AS agg2#37, cast(cs_coupon_amt#8 as decimal(12,2)) AS agg3#38, cast(cs_sales_price#7 as decimal(12,2)) AS agg4#39, cast(cs_net_profit#9 as decimal(12,2)) AS agg5#40, cast(c_birth_year#26 as decimal(12,2)) AS agg6#41, cast(cd_dep_count#13 as decimal(12,2)) AS agg7#42] -Input [11]: [cs_bill_customer_sk#2, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, i_item_id#19, c_customer_sk#22, c_birth_year#26, ca_country#30] +Output [9]: [i_item_id#19, ca_country#33, cast(cs_quantity#5 as decimal(12,2)) AS agg1#36, cast(cs_list_price#6 as decimal(12,2)) AS agg2#37, cast(cs_coupon_amt#8 as decimal(12,2)) AS agg3#38, cast(cs_sales_price#7 as decimal(12,2)) AS agg4#39, cast(cs_net_profit#9 as decimal(12,2)) AS agg5#40, cast(c_birth_year#26 as decimal(12,2)) AS agg6#41, cast(cd_dep_count#13 as decimal(12,2)) AS agg7#42] +Input [11]: [cs_bill_customer_sk#2, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, i_item_id#19, c_customer_sk#22, c_birth_year#26, ca_country#33] (100) HashAggregate [codegen id : 41] -Input [9]: [i_item_id#19, ca_country#30, agg1#36, agg2#37, agg3#38, agg4#39, agg5#40, agg6#41, agg7#42] -Keys [2]: [i_item_id#19, ca_country#30] +Input [9]: [i_item_id#19, ca_country#33, agg1#36, agg2#37, agg3#38, agg4#39, agg5#40, agg6#41, agg7#42] +Keys [2]: [i_item_id#19, ca_country#33] Functions [7]: [partial_avg(agg1#36), partial_avg(agg2#37), partial_avg(agg3#38), partial_avg(agg4#39), partial_avg(agg5#40), partial_avg(agg6#41), partial_avg(agg7#42)] Aggregate Attributes [14]: [sum#138, count#139, sum#140, count#141, sum#142, count#143, sum#144, count#145, sum#146, count#147, sum#148, count#149, sum#150, count#151] -Results [16]: [i_item_id#19, ca_country#30, sum#152, count#153, sum#154, count#155, sum#156, count#157, sum#158, count#159, sum#160, count#161, sum#162, count#163, sum#164, count#165] +Results [16]: [i_item_id#19, ca_country#33, sum#152, count#153, sum#154, count#155, sum#156, count#157, sum#158, count#159, sum#160, count#161, sum#162, count#163, sum#164, count#165] (101) Exchange -Input [16]: [i_item_id#19, ca_country#30, sum#152, count#153, sum#154, count#155, sum#156, count#157, sum#158, count#159, sum#160, count#161, sum#162, count#163, sum#164, count#165] -Arguments: hashpartitioning(i_item_id#19, ca_country#30, 5), true, [id=#166] +Input [16]: [i_item_id#19, ca_country#33, sum#152, count#153, sum#154, count#155, sum#156, count#157, sum#158, count#159, sum#160, count#161, sum#162, count#163, sum#164, count#165] +Arguments: hashpartitioning(i_item_id#19, ca_country#33, 5), ENSURE_REQUIREMENTS, [id=#166] (102) HashAggregate [codegen id : 42] -Input [16]: [i_item_id#19, ca_country#30, sum#152, count#153, sum#154, count#155, sum#156, count#157, sum#158, count#159, sum#160, count#161, sum#162, count#163, sum#164, count#165] -Keys [2]: [i_item_id#19, ca_country#30] +Input [16]: [i_item_id#19, ca_country#33, sum#152, count#153, sum#154, count#155, sum#156, count#157, sum#158, count#159, sum#160, count#161, sum#162, count#163, sum#164, count#165] +Keys [2]: [i_item_id#19, ca_country#33] Functions [7]: [avg(agg1#36), avg(agg2#37), avg(agg3#38), avg(agg4#39), avg(agg5#40), avg(agg6#41), avg(agg7#42)] Aggregate Attributes [7]: [avg(agg1#36)#167, avg(agg2#37)#168, avg(agg3#38)#169, avg(agg4#39)#170, avg(agg5#40)#171, avg(agg6#41)#172, avg(agg7#42)#173] -Results [11]: [i_item_id#19, ca_country#30, null AS ca_state#174, null AS county#175, avg(agg1#36)#167 AS agg1#176, avg(agg2#37)#168 AS agg2#177, avg(agg3#38)#169 AS agg3#178, avg(agg4#39)#170 AS agg4#179, avg(agg5#40)#171 AS agg5#180, avg(agg6#41)#172 AS agg6#181, avg(agg7#42)#173 AS agg7#182] +Results [11]: [i_item_id#19, ca_country#33, null AS ca_state#174, null AS county#175, avg(agg1#36)#167 AS agg1#176, avg(agg2#37)#168 AS agg2#177, avg(agg3#38)#169 AS agg3#178, avg(agg4#39)#170 AS agg4#179, avg(agg5#40)#171 AS agg5#180, avg(agg6#41)#172 AS agg6#181, avg(agg7#42)#173 AS agg7#182] (103) Scan parquet default.catalog_sales Output [9]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_bill_cdemo_sk#3, cs_item_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9] @@ -674,35 +674,35 @@ Output [4]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_bi Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_month#25, c_birth_year#26] (116) Scan parquet default.customer_address -Output [2]: [ca_address_sk#27, ca_state#29] +Output [2]: [ca_address_sk#30, ca_state#32] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_state, [ND,WI,AL,NC,OK,MS,TN]), IsNotNull(ca_address_sk)] ReadSchema: struct (117) ColumnarToRow [codegen id : 45] -Input [2]: [ca_address_sk#27, ca_state#29] +Input [2]: [ca_address_sk#30, ca_state#32] (118) Filter [codegen id : 45] -Input [2]: [ca_address_sk#27, ca_state#29] -Condition : (ca_state#29 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#27)) +Input [2]: [ca_address_sk#30, ca_state#32] +Condition : (ca_state#32 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#30)) (119) Project [codegen id : 45] -Output [1]: [ca_address_sk#27] -Input [2]: [ca_address_sk#27, ca_state#29] +Output [1]: [ca_address_sk#30] +Input [2]: [ca_address_sk#30, ca_state#32] (120) BroadcastExchange -Input [1]: [ca_address_sk#27] +Input [1]: [ca_address_sk#30] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#183] (121) BroadcastHashJoin [codegen id : 46] Left keys [1]: [c_current_addr_sk#24] -Right keys [1]: [ca_address_sk#27] +Right keys [1]: [ca_address_sk#30] Join condition: None (122) Project [codegen id : 46] Output [3]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26] -Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_year#26, ca_address_sk#27] +Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_year#26, ca_address_sk#30] (123) BroadcastExchange Input [3]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26] @@ -765,7 +765,7 @@ Results [15]: [i_item_id#19, sum#201, count#202, sum#203, count#204, sum#205, co (136) Exchange Input [15]: [i_item_id#19, sum#201, count#202, sum#203, count#204, sum#205, count#206, sum#207, count#208, sum#209, count#210, sum#211, count#212, sum#213, count#214] -Arguments: hashpartitioning(i_item_id#19, 5), true, [id=#215] +Arguments: hashpartitioning(i_item_id#19, 5), ENSURE_REQUIREMENTS, [id=#215] (137) HashAggregate [codegen id : 50] Input [15]: [i_item_id#19, sum#201, count#202, sum#203, count#204, sum#205, count#206, sum#207, count#208, sum#209, count#210, sum#211, count#212, sum#213, count#214] @@ -860,7 +860,7 @@ Results [14]: [sum#248, count#249, sum#250, count#251, sum#252, count#253, sum#2 (157) Exchange Input [14]: [sum#248, count#249, sum#250, count#251, sum#252, count#253, sum#254, count#255, sum#256, count#257, sum#258, count#259, sum#260, count#261] -Arguments: SinglePartition, true, [id=#262] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#262] (158) HashAggregate [codegen id : 58] Input [14]: [sum#248, count#249, sum#250, count#251, sum#252, count#253, sum#254, count#255, sum#256, count#257, sum#258, count#259, sum#260, count#261] @@ -872,6 +872,6 @@ Results [11]: [null AS i_item_id#270, null AS ca_country#271, null AS ca_state#2 (159) Union (160) TakeOrderedAndProject -Input [11]: [i_item_id#19, ca_country#30, ca_state#29, ca_county#28, agg1#79, agg2#80, agg3#81, agg4#82, agg5#83, agg6#84, agg7#85] -Arguments: 100, [ca_country#30 ASC NULLS FIRST, ca_state#29 ASC NULLS FIRST, ca_county#28 ASC NULLS FIRST, i_item_id#19 ASC NULLS FIRST], [i_item_id#19, ca_country#30, ca_state#29, ca_county#28, agg1#79, agg2#80, agg3#81, agg4#82, agg5#83, agg6#84, agg7#85] +Input [11]: [i_item_id#19, ca_country#33, ca_state#32, ca_county#31, agg1#79, agg2#80, agg3#81, agg4#82, agg5#83, agg6#84, agg7#85] +Arguments: 100, [ca_country#33 ASC NULLS FIRST, ca_state#32 ASC NULLS FIRST, ca_county#31 ASC NULLS FIRST, i_item_id#19 ASC NULLS FIRST], [i_item_id#19, ca_country#33, ca_state#32, ca_county#31, agg1#79, agg2#80, agg3#81, agg4#82, agg5#83, agg6#84, agg7#85] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.sf100/simplified.txt index 5514e335f1b51..4566929712713 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.sf100/simplified.txt @@ -54,37 +54,37 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag Exchange [c_customer_sk] #6 WholeStageCodegen (11) Project [c_customer_sk,c_birth_year,ca_county,ca_state,ca_country] - SortMergeJoin [c_current_cdemo_sk,cd_demo_sk] - InputAdapter - WholeStageCodegen (8) - Sort [c_current_cdemo_sk] - InputAdapter - Exchange [c_current_cdemo_sk] #7 - WholeStageCodegen (7) - Project [c_customer_sk,c_current_cdemo_sk,c_birth_year,ca_county,ca_state,ca_country] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [c_customer_sk,c_current_addr_sk,c_birth_year] + SortMergeJoin [c_current_cdemo_sk,cd_demo_sk] + InputAdapter + WholeStageCodegen (7) + Sort [c_current_cdemo_sk] + InputAdapter + Exchange [c_current_cdemo_sk] #7 + WholeStageCodegen (6) Project [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] Filter [c_birth_month,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] ColumnarToRow InputAdapter Scan parquet default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (6) - Filter [ca_state,ca_address_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] + InputAdapter + WholeStageCodegen (9) + Sort [cd_demo_sk] + InputAdapter + Exchange [cd_demo_sk] #8 + WholeStageCodegen (8) + Filter [cd_demo_sk] + ColumnarToRow + InputAdapter + Scan parquet default.customer_demographics [cd_demo_sk] InputAdapter - WholeStageCodegen (10) - Sort [cd_demo_sk] - InputAdapter - Exchange [cd_demo_sk] #9 - WholeStageCodegen (9) - Filter [cd_demo_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer_demographics [cd_demo_sk] + BroadcastExchange #9 + WholeStageCodegen (10) + Filter [ca_state,ca_address_sk] + ColumnarToRow + InputAdapter + Scan parquet default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] WholeStageCodegen (28) HashAggregate [i_item_id,ca_country,ca_state,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7),county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] InputAdapter @@ -130,7 +130,7 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag WholeStageCodegen (24) Sort [cd_demo_sk] InputAdapter - ReusedExchange [cd_demo_sk] #9 + ReusedExchange [cd_demo_sk] #8 WholeStageCodegen (42) HashAggregate [i_item_id,ca_country,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7),ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] InputAdapter @@ -177,7 +177,7 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag WholeStageCodegen (38) Sort [cd_demo_sk] InputAdapter - ReusedExchange [cd_demo_sk] #9 + ReusedExchange [cd_demo_sk] #8 WholeStageCodegen (50) HashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7),ca_country,ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] InputAdapter 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 a7f328537b7ac..04ff822b1ce52 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 @@ -11,60 +11,60 @@ TakeOrderedAndProject (79) : +- * BroadcastHashJoin LeftOuter BuildRight (65) : :- * Project (60) : : +- * SortMergeJoin Inner (59) - : : :- * Sort (47) - : : : +- Exchange (46) - : : : +- * Project (45) - : : : +- * BroadcastHashJoin Inner BuildRight (44) - : : : :- * Project (32) - : : : : +- * SortMergeJoin Inner (31) - : : : : :- * Sort (25) - : : : : : +- Exchange (24) - : : : : : +- * Project (23) - : : : : : +- * BroadcastHashJoin Inner BuildRight (22) - : : : : : :- * Project (17) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (16) - : : : : : : :- * Project (10) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : : : : :- * Filter (3) - : : : : : : : : +- * ColumnarToRow (2) - : : : : : : : : +- Scan parquet default.catalog_sales (1) - : : : : : : : +- BroadcastExchange (8) - : : : : : : : +- * Project (7) - : : : : : : : +- * Filter (6) - : : : : : : : +- * ColumnarToRow (5) - : : : : : : : +- Scan parquet default.household_demographics (4) - : : : : : : +- BroadcastExchange (15) - : : : : : : +- * Project (14) - : : : : : : +- * Filter (13) - : : : : : : +- * ColumnarToRow (12) - : : : : : : +- Scan parquet default.customer_demographics (11) - : : : : : +- BroadcastExchange (21) - : : : : : +- * Filter (20) - : : : : : +- * ColumnarToRow (19) - : : : : : +- Scan parquet default.date_dim (18) - : : : : +- * Sort (30) - : : : : +- Exchange (29) - : : : : +- * Filter (28) - : : : : +- * ColumnarToRow (27) - : : : : +- Scan parquet default.item (26) - : : : +- BroadcastExchange (43) - : : : +- * Project (42) - : : : +- * BroadcastHashJoin Inner BuildLeft (41) - : : : :- BroadcastExchange (37) - : : : : +- * Project (36) - : : : : +- * Filter (35) - : : : : +- * ColumnarToRow (34) - : : : : +- Scan parquet default.date_dim (33) - : : : +- * Filter (40) - : : : +- * ColumnarToRow (39) - : : : +- Scan parquet default.date_dim (38) + : : :- * Sort (34) + : : : +- Exchange (33) + : : : +- * Project (32) + : : : +- * SortMergeJoin Inner (31) + : : : :- * Sort (25) + : : : : +- Exchange (24) + : : : : +- * Project (23) + : : : : +- * BroadcastHashJoin Inner BuildRight (22) + : : : : :- * Project (17) + : : : : : +- * BroadcastHashJoin Inner BuildRight (16) + : : : : : :- * Project (10) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : : : : :- * Filter (3) + : : : : : : : +- * ColumnarToRow (2) + : : : : : : : +- Scan parquet default.catalog_sales (1) + : : : : : : +- BroadcastExchange (8) + : : : : : : +- * Project (7) + : : : : : : +- * Filter (6) + : : : : : : +- * ColumnarToRow (5) + : : : : : : +- Scan parquet default.household_demographics (4) + : : : : : +- BroadcastExchange (15) + : : : : : +- * Project (14) + : : : : : +- * Filter (13) + : : : : : +- * ColumnarToRow (12) + : : : : : +- Scan parquet default.customer_demographics (11) + : : : : +- BroadcastExchange (21) + : : : : +- * Filter (20) + : : : : +- * ColumnarToRow (19) + : : : : +- Scan parquet default.date_dim (18) + : : : +- * Sort (30) + : : : +- Exchange (29) + : : : +- * Filter (28) + : : : +- * ColumnarToRow (27) + : : : +- Scan parquet default.item (26) : : +- * Sort (58) : : +- Exchange (57) : : +- * Project (56) : : +- * BroadcastHashJoin Inner BuildRight (55) - : : :- * Filter (50) - : : : +- * ColumnarToRow (49) - : : : +- Scan parquet default.inventory (48) + : : :- * Project (50) + : : : +- * BroadcastHashJoin Inner BuildLeft (49) + : : : :- BroadcastExchange (45) + : : : : +- * Project (44) + : : : : +- * BroadcastHashJoin Inner BuildLeft (43) + : : : : :- BroadcastExchange (39) + : : : : : +- * Project (38) + : : : : : +- * Filter (37) + : : : : : +- * ColumnarToRow (36) + : : : : : +- Scan parquet default.date_dim (35) + : : : : +- * Filter (42) + : : : : +- * ColumnarToRow (41) + : : : : +- Scan parquet default.date_dim (40) + : : : +- * Filter (48) + : : : +- * ColumnarToRow (47) + : : : +- Scan parquet default.inventory (46) : : +- BroadcastExchange (54) : : +- * Filter (53) : : +- * ColumnarToRow (52) @@ -185,7 +185,7 @@ Input [8]: [cs_sold_date_sk#1, cs_ship_date_sk#2, cs_item_sk#5, cs_promo_sk#6, c (24) Exchange Input [6]: [cs_sold_date_sk#1, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, d_date#16] -Arguments: hashpartitioning(cs_item_sk#5, 5), true, [id=#18] +Arguments: hashpartitioning(cs_item_sk#5, 5), ENSURE_REQUIREMENTS, [id=#18] (25) Sort [codegen id : 5] Input [6]: [cs_sold_date_sk#1, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, d_date#16] @@ -207,101 +207,101 @@ Condition : isnotnull(i_item_sk#19) (29) Exchange Input [2]: [i_item_sk#19, i_item_desc#20] -Arguments: hashpartitioning(i_item_sk#19, 5), true, [id=#21] +Arguments: hashpartitioning(i_item_sk#19, 5), ENSURE_REQUIREMENTS, [id=#21] (30) Sort [codegen id : 7] Input [2]: [i_item_sk#19, i_item_desc#20] Arguments: [i_item_sk#19 ASC NULLS FIRST], false, 0 -(31) SortMergeJoin [codegen id : 10] +(31) SortMergeJoin [codegen id : 8] Left keys [1]: [cs_item_sk#5] Right keys [1]: [i_item_sk#19] Join condition: None -(32) Project [codegen id : 10] +(32) Project [codegen id : 8] Output [7]: [cs_sold_date_sk#1, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, d_date#16, i_item_desc#20] Input [8]: [cs_sold_date_sk#1, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, d_date#16, i_item_sk#19, i_item_desc#20] -(33) Scan parquet default.date_dim -Output [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] +(33) Exchange +Input [7]: [cs_sold_date_sk#1, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, d_date#16, i_item_desc#20] +Arguments: hashpartitioning(cs_item_sk#5, cs_sold_date_sk#1, 5), ENSURE_REQUIREMENTS, [id=#22] + +(34) Sort [codegen id : 9] +Input [7]: [cs_sold_date_sk#1, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, d_date#16, i_item_desc#20] +Arguments: [cs_item_sk#5 ASC NULLS FIRST, cs_sold_date_sk#1 ASC NULLS FIRST], false, 0 + +(35) Scan parquet default.date_dim +Output [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_year#26] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk), IsNotNull(d_week_seq), IsNotNull(d_date)] ReadSchema: struct -(34) ColumnarToRow [codegen id : 8] -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] +(36) ColumnarToRow [codegen id : 10] +Input [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_year#26] -(35) Filter [codegen id : 8] -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -Condition : ((((isnotnull(d_year#25) AND (d_year#25 = 2001)) AND isnotnull(d_date_sk#22)) AND isnotnull(d_week_seq#24)) AND isnotnull(d_date#23)) +(37) Filter [codegen id : 10] +Input [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_year#26] +Condition : ((((isnotnull(d_year#26) AND (d_year#26 = 2001)) AND isnotnull(d_date_sk#23)) AND isnotnull(d_week_seq#25)) AND isnotnull(d_date#24)) -(36) Project [codegen id : 8] -Output [3]: [d_date_sk#22, d_date#23, d_week_seq#24] -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] +(38) Project [codegen id : 10] +Output [3]: [d_date_sk#23, d_date#24, d_week_seq#25] +Input [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_year#26] -(37) BroadcastExchange -Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[2, int, true] as bigint)),false), [id=#26] +(39) BroadcastExchange +Input [3]: [d_date_sk#23, d_date#24, d_week_seq#25] +Arguments: HashedRelationBroadcastMode(List(cast(input[2, int, true] as bigint)),false), [id=#27] -(38) Scan parquet default.date_dim -Output [2]: [d_date_sk#27, d_week_seq#28] +(40) Scan parquet default.date_dim +Output [2]: [d_date_sk#28, d_week_seq#29] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(39) ColumnarToRow -Input [2]: [d_date_sk#27, d_week_seq#28] +(41) ColumnarToRow +Input [2]: [d_date_sk#28, d_week_seq#29] -(40) Filter -Input [2]: [d_date_sk#27, d_week_seq#28] -Condition : (isnotnull(d_week_seq#28) AND isnotnull(d_date_sk#27)) +(42) Filter +Input [2]: [d_date_sk#28, d_week_seq#29] +Condition : (isnotnull(d_week_seq#29) AND isnotnull(d_date_sk#28)) -(41) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [d_week_seq#24] -Right keys [1]: [d_week_seq#28] +(43) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [d_week_seq#25] +Right keys [1]: [d_week_seq#29] Join condition: None -(42) Project [codegen id : 9] -Output [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_date_sk#27] -Input [5]: [d_date_sk#22, d_date#23, d_week_seq#24, d_date_sk#27, d_week_seq#28] - -(43) BroadcastExchange -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_date_sk#27] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#29] - -(44) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#1] -Right keys [1]: [d_date_sk#22] -Join condition: (d_date#16 > d_date#23 + 5 days) - -(45) Project [codegen id : 10] -Output [7]: [cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, i_item_desc#20, d_week_seq#24, d_date_sk#27] -Input [11]: [cs_sold_date_sk#1, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, d_date#16, i_item_desc#20, d_date_sk#22, d_date#23, d_week_seq#24, d_date_sk#27] +(44) Project [codegen id : 11] +Output [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_date_sk#28] +Input [5]: [d_date_sk#23, d_date#24, d_week_seq#25, d_date_sk#28, d_week_seq#29] -(46) Exchange -Input [7]: [cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, i_item_desc#20, d_week_seq#24, d_date_sk#27] -Arguments: hashpartitioning(cs_item_sk#5, d_date_sk#27, 5), true, [id=#30] +(45) BroadcastExchange +Input [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_date_sk#28] +Arguments: HashedRelationBroadcastMode(List(cast(input[3, int, true] as bigint)),false), [id=#30] -(47) Sort [codegen id : 11] -Input [7]: [cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, i_item_desc#20, d_week_seq#24, d_date_sk#27] -Arguments: [cs_item_sk#5 ASC NULLS FIRST, d_date_sk#27 ASC NULLS FIRST], false, 0 - -(48) Scan parquet default.inventory +(46) Scan parquet default.inventory Output [4]: [inv_date_sk#31, inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34] Batched: true Location [not included in comparison]/{warehouse_dir}/inventory] PushedFilters: [IsNotNull(inv_quantity_on_hand), IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk), IsNotNull(inv_date_sk)] ReadSchema: struct -(49) ColumnarToRow [codegen id : 13] +(47) ColumnarToRow Input [4]: [inv_date_sk#31, inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34] -(50) Filter [codegen id : 13] +(48) Filter Input [4]: [inv_date_sk#31, inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34] Condition : (((isnotnull(inv_quantity_on_hand#34) AND isnotnull(inv_item_sk#32)) AND isnotnull(inv_warehouse_sk#33)) AND isnotnull(inv_date_sk#31)) +(49) BroadcastHashJoin [codegen id : 13] +Left keys [1]: [d_date_sk#28] +Right keys [1]: [inv_date_sk#31] +Join condition: None + +(50) Project [codegen id : 13] +Output [6]: [d_date_sk#23, d_date#24, d_week_seq#25, inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34] +Input [8]: [d_date_sk#23, d_date#24, d_week_seq#25, d_date_sk#28, inv_date_sk#31, inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34] + (51) Scan parquet default.warehouse Output [2]: [w_warehouse_sk#35, w_warehouse_name#36] Batched: true @@ -326,25 +326,25 @@ Right keys [1]: [w_warehouse_sk#35] Join condition: None (56) Project [codegen id : 13] -Output [4]: [inv_date_sk#31, inv_item_sk#32, inv_quantity_on_hand#34, w_warehouse_name#36] -Input [6]: [inv_date_sk#31, inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34, w_warehouse_sk#35, w_warehouse_name#36] +Output [6]: [d_date_sk#23, d_date#24, d_week_seq#25, inv_item_sk#32, inv_quantity_on_hand#34, w_warehouse_name#36] +Input [8]: [d_date_sk#23, d_date#24, d_week_seq#25, inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34, w_warehouse_sk#35, w_warehouse_name#36] (57) Exchange -Input [4]: [inv_date_sk#31, inv_item_sk#32, inv_quantity_on_hand#34, w_warehouse_name#36] -Arguments: hashpartitioning(inv_item_sk#32, inv_date_sk#31, 5), true, [id=#38] +Input [6]: [d_date_sk#23, d_date#24, d_week_seq#25, inv_item_sk#32, inv_quantity_on_hand#34, w_warehouse_name#36] +Arguments: hashpartitioning(inv_item_sk#32, d_date_sk#23, 5), ENSURE_REQUIREMENTS, [id=#38] (58) Sort [codegen id : 14] -Input [4]: [inv_date_sk#31, inv_item_sk#32, inv_quantity_on_hand#34, w_warehouse_name#36] -Arguments: [inv_item_sk#32 ASC NULLS FIRST, inv_date_sk#31 ASC NULLS FIRST], false, 0 +Input [6]: [d_date_sk#23, d_date#24, d_week_seq#25, inv_item_sk#32, inv_quantity_on_hand#34, w_warehouse_name#36] +Arguments: [inv_item_sk#32 ASC NULLS FIRST, d_date_sk#23 ASC NULLS FIRST], false, 0 (59) SortMergeJoin [codegen id : 16] -Left keys [2]: [cs_item_sk#5, d_date_sk#27] -Right keys [2]: [inv_item_sk#32, inv_date_sk#31] -Join condition: (inv_quantity_on_hand#34 < cs_quantity#8) +Left keys [2]: [cs_item_sk#5, cs_sold_date_sk#1] +Right keys [2]: [inv_item_sk#32, d_date_sk#23] +Join condition: ((inv_quantity_on_hand#34 < cs_quantity#8) AND (d_date#16 > d_date#24 + 5 days)) (60) Project [codegen id : 16] -Output [6]: [cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#24] -Input [11]: [cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, i_item_desc#20, d_week_seq#24, d_date_sk#27, inv_date_sk#31, inv_item_sk#32, inv_quantity_on_hand#34, w_warehouse_name#36] +Output [6]: [cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#25] +Input [13]: [cs_sold_date_sk#1, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, d_date#16, i_item_desc#20, d_date_sk#23, d_date#24, d_week_seq#25, inv_item_sk#32, inv_quantity_on_hand#34, w_warehouse_name#36] (61) Scan parquet default.promotion Output [1]: [p_promo_sk#39] @@ -370,15 +370,15 @@ Right keys [1]: [p_promo_sk#39] Join condition: None (66) Project [codegen id : 16] -Output [5]: [cs_item_sk#5, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#24] -Input [7]: [cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#24, p_promo_sk#39] +Output [5]: [cs_item_sk#5, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#25] +Input [7]: [cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#25, p_promo_sk#39] (67) Exchange -Input [5]: [cs_item_sk#5, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#24] -Arguments: hashpartitioning(cs_item_sk#5, cs_order_number#7, 5), true, [id=#41] +Input [5]: [cs_item_sk#5, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#25] +Arguments: hashpartitioning(cs_item_sk#5, cs_order_number#7, 5), ENSURE_REQUIREMENTS, [id=#41] (68) Sort [codegen id : 17] -Input [5]: [cs_item_sk#5, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#24] +Input [5]: [cs_item_sk#5, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#25] Arguments: [cs_item_sk#5 ASC NULLS FIRST, cs_order_number#7 ASC NULLS FIRST], false, 0 (69) Scan parquet default.catalog_returns @@ -397,7 +397,7 @@ Condition : (isnotnull(cr_item_sk#42) AND isnotnull(cr_order_number#43)) (72) Exchange Input [2]: [cr_item_sk#42, cr_order_number#43] -Arguments: hashpartitioning(cr_item_sk#42, cr_order_number#43, 5), true, [id=#44] +Arguments: hashpartitioning(cr_item_sk#42, cr_order_number#43, 5), ENSURE_REQUIREMENTS, [id=#44] (73) Sort [codegen id : 19] Input [2]: [cr_item_sk#42, cr_order_number#43] @@ -409,28 +409,28 @@ Right keys [2]: [cr_item_sk#42, cr_order_number#43] Join condition: None (75) Project [codegen id : 20] -Output [3]: [w_warehouse_name#36, i_item_desc#20, d_week_seq#24] -Input [7]: [cs_item_sk#5, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#24, cr_item_sk#42, cr_order_number#43] +Output [3]: [w_warehouse_name#36, i_item_desc#20, d_week_seq#25] +Input [7]: [cs_item_sk#5, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#25, cr_item_sk#42, cr_order_number#43] (76) HashAggregate [codegen id : 20] -Input [3]: [w_warehouse_name#36, i_item_desc#20, d_week_seq#24] -Keys [3]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#24] +Input [3]: [w_warehouse_name#36, i_item_desc#20, d_week_seq#25] +Keys [3]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#25] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#45] -Results [4]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#24, count#46] +Results [4]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#25, count#46] (77) Exchange -Input [4]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#24, count#46] -Arguments: hashpartitioning(i_item_desc#20, w_warehouse_name#36, d_week_seq#24, 5), true, [id=#47] +Input [4]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#25, count#46] +Arguments: hashpartitioning(i_item_desc#20, w_warehouse_name#36, d_week_seq#25, 5), ENSURE_REQUIREMENTS, [id=#47] (78) HashAggregate [codegen id : 21] -Input [4]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#24, count#46] -Keys [3]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#24] +Input [4]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#25, count#46] +Keys [3]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#25] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#48] -Results [6]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#24, count(1)#48 AS no_promo#49, count(1)#48 AS promo#50, count(1)#48 AS total_cnt#51] +Results [6]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#25, count(1)#48 AS no_promo#49, count(1)#48 AS promo#50, count(1)#48 AS total_cnt#51] (79) TakeOrderedAndProject -Input [6]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#24, no_promo#49, promo#50, total_cnt#51] -Arguments: 100, [total_cnt#51 DESC NULLS LAST, i_item_desc#20 ASC NULLS FIRST, w_warehouse_name#36 ASC NULLS FIRST, d_week_seq#24 ASC NULLS FIRST], [i_item_desc#20, w_warehouse_name#36, d_week_seq#24, no_promo#49, promo#50, total_cnt#51] +Input [6]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#25, no_promo#49, promo#50, total_cnt#51] +Arguments: 100, [total_cnt#51 DESC NULLS LAST, i_item_desc#20 ASC NULLS FIRST, w_warehouse_name#36 ASC NULLS FIRST, d_week_seq#25 ASC NULLS FIRST], [i_item_desc#20, w_warehouse_name#36, d_week_seq#25, no_promo#49, promo#50, total_cnt#51] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/simplified.txt index 918508787c4b0..b88505ad7b9bc 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/simplified.txt @@ -16,95 +16,95 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom Project [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] BroadcastHashJoin [cs_promo_sk,p_promo_sk] Project [cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - SortMergeJoin [cs_item_sk,d_date_sk,inv_item_sk,inv_date_sk,inv_quantity_on_hand,cs_quantity] + SortMergeJoin [cs_item_sk,cs_sold_date_sk,inv_item_sk,d_date_sk,inv_quantity_on_hand,cs_quantity,d_date,d_date] InputAdapter - WholeStageCodegen (11) - Sort [cs_item_sk,d_date_sk] + WholeStageCodegen (9) + Sort [cs_item_sk,cs_sold_date_sk] InputAdapter - Exchange [cs_item_sk,d_date_sk] #3 - WholeStageCodegen (10) - Project [cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,i_item_desc,d_week_seq,d_date_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk,d_date,d_date] - Project [cs_sold_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,d_date,i_item_desc] - SortMergeJoin [cs_item_sk,i_item_sk] - InputAdapter - WholeStageCodegen (5) - Sort [cs_item_sk] - InputAdapter - Exchange [cs_item_sk] #4 - WholeStageCodegen (4) - Project [cs_sold_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,d_date] - BroadcastHashJoin [cs_ship_date_sk,d_date_sk] - Project [cs_sold_date_sk,cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity] - BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - Project [cs_sold_date_sk,cs_ship_date_sk,cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity] - BroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk] - Filter [cs_quantity,cs_item_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_sold_date_sk,cs_ship_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity] + Exchange [cs_item_sk,cs_sold_date_sk] #3 + WholeStageCodegen (8) + Project [cs_sold_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,d_date,i_item_desc] + SortMergeJoin [cs_item_sk,i_item_sk] + InputAdapter + WholeStageCodegen (5) + Sort [cs_item_sk] + InputAdapter + Exchange [cs_item_sk] #4 + WholeStageCodegen (4) + Project [cs_sold_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,d_date] + BroadcastHashJoin [cs_ship_date_sk,d_date_sk] + Project [cs_sold_date_sk,cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity] + BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + Project [cs_sold_date_sk,cs_ship_date_sk,cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity] + BroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk] + Filter [cs_quantity,cs_item_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_sold_date_sk,cs_ship_date_sk] + ColumnarToRow InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - Project [hd_demo_sk] - Filter [hd_buy_potential,hd_demo_sk] - ColumnarToRow - InputAdapter - Scan parquet default.household_demographics [hd_demo_sk,hd_buy_potential] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (2) - Project [cd_demo_sk] - Filter [cd_marital_status,cd_demo_sk] + BroadcastExchange #5 + WholeStageCodegen (1) + Project [hd_demo_sk] + Filter [hd_buy_potential,hd_demo_sk] ColumnarToRow InputAdapter - Scan parquet default.customer_demographics [cd_demo_sk,cd_marital_status] + Scan parquet default.household_demographics [hd_demo_sk,hd_buy_potential] InputAdapter - BroadcastExchange #7 - WholeStageCodegen (3) - Filter [d_date,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date] - InputAdapter - WholeStageCodegen (7) - Sort [i_item_sk] - InputAdapter - Exchange [i_item_sk] #8 - WholeStageCodegen (6) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_item_desc] + BroadcastExchange #6 + WholeStageCodegen (2) + Project [cd_demo_sk] + Filter [cd_marital_status,cd_demo_sk] + ColumnarToRow + InputAdapter + Scan parquet default.customer_demographics [cd_demo_sk,cd_marital_status] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (3) + Filter [d_date,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_date] InputAdapter - BroadcastExchange #9 - WholeStageCodegen (9) - Project [d_date_sk,d_date,d_week_seq,d_date_sk] - BroadcastHashJoin [d_week_seq,d_week_seq] - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (8) - Project [d_date_sk,d_date,d_week_seq] - Filter [d_year,d_date_sk,d_week_seq,d_date] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - Filter [d_week_seq,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_week_seq] + WholeStageCodegen (7) + Sort [i_item_sk] + InputAdapter + Exchange [i_item_sk] #8 + WholeStageCodegen (6) + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_item_desc] InputAdapter WholeStageCodegen (14) - Sort [inv_item_sk,inv_date_sk] + Sort [inv_item_sk,d_date_sk] InputAdapter - Exchange [inv_item_sk,inv_date_sk] #11 + Exchange [inv_item_sk,d_date_sk] #9 WholeStageCodegen (13) - Project [inv_date_sk,inv_item_sk,inv_quantity_on_hand,w_warehouse_name] + Project [d_date_sk,d_date,d_week_seq,inv_item_sk,inv_quantity_on_hand,w_warehouse_name] BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Filter [inv_quantity_on_hand,inv_item_sk,inv_warehouse_sk,inv_date_sk] - ColumnarToRow + Project [d_date_sk,d_date,d_week_seq,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] + BroadcastHashJoin [d_date_sk,inv_date_sk] InputAdapter - Scan parquet default.inventory [inv_date_sk,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] + BroadcastExchange #10 + WholeStageCodegen (11) + Project [d_date_sk,d_date,d_week_seq,d_date_sk] + BroadcastHashJoin [d_week_seq,d_week_seq] + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (10) + Project [d_date_sk,d_date,d_week_seq] + Filter [d_year,d_date_sk,d_week_seq,d_date] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_date,d_week_seq,d_year] + Filter [d_week_seq,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_week_seq] + Filter [inv_quantity_on_hand,inv_item_sk,inv_warehouse_sk,inv_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.inventory [inv_date_sk,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] InputAdapter BroadcastExchange #12 WholeStageCodegen (12) From 122f8f0fdb0fdc87a5970f4b39938a0496bd4b4b Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Tue, 5 Jan 2021 07:30:59 +0000 Subject: [PATCH 03/29] [SPARK-33919][SQL][TESTS] Unify v1 and v2 SHOW NAMESPACES tests ### What changes were proposed in this pull request? 1. Port DS V2 tests from `DataSourceV2SQLSuite` to the base test suite `ShowNamespacesSuiteBase` to run those tests for v1 catalogs. 2. Port DS v1 tests from `DDLSuite` to `ShowNamespacesSuiteBase` to run the tests for v2 catalogs too. ### Why are the changes needed? To improve test coverage. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? By running new test suites: ``` $ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *ShowNamespacesSuite" ``` Closes #30937 from MaxGekk/unify-show-namespaces-tests. Authored-by: Max Gekk Signed-off-by: Wenchen Fan --- .../sql/catalyst/parser/DDLParserSuite.scala | 34 ----- .../sql/connector/DataSourceV2SQLSuite.scala | 89 ------------ .../sql/execution/command/DDLSuite.scala | 30 ---- .../command/ShowNamespacesParserSuite.scala | 70 ++++++++++ .../command/ShowNamespacesSuiteBase.scala | 131 ++++++++++++++++++ .../command/v1/ShowNamespacesSuite.scala | 60 ++++++++ .../command/v2/ShowNamespacesSuite.scala | 72 ++++++++++ .../command/ShowNamespacesSuite.scala | 43 ++++++ 8 files changed, 376 insertions(+), 153 deletions(-) create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowNamespacesParserSuite.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowNamespacesSuiteBase.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowNamespacesSuite.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowNamespacesSuite.scala create mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/ShowNamespacesSuite.scala diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala index 9ec22a982a588..4978a3a6653c4 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala @@ -1807,40 +1807,6 @@ class DDLParserSuite extends AnalysisTest { UnresolvedNamespace(Seq("a", "b", "c")), "/home/user/db")) } - test("show databases: basic") { - comparePlans( - parsePlan("SHOW DATABASES"), - ShowNamespaces(UnresolvedNamespace(Seq.empty[String]), None)) - comparePlans( - parsePlan("SHOW DATABASES LIKE 'defau*'"), - ShowNamespaces(UnresolvedNamespace(Seq.empty[String]), Some("defau*"))) - } - - test("show databases: FROM/IN operator is not allowed") { - def verify(sql: String): Unit = { - val exc = intercept[ParseException] { parsePlan(sql) } - assert(exc.getMessage.contains("FROM/IN operator is not allowed in SHOW DATABASES")) - } - - verify("SHOW DATABASES FROM testcat.ns1.ns2") - verify("SHOW DATABASES IN testcat.ns1.ns2") - } - - test("show namespaces") { - comparePlans( - parsePlan("SHOW NAMESPACES"), - ShowNamespaces(UnresolvedNamespace(Seq.empty[String]), None)) - comparePlans( - parsePlan("SHOW NAMESPACES FROM testcat.ns1.ns2"), - ShowNamespaces(UnresolvedNamespace(Seq("testcat", "ns1", "ns2")), None)) - comparePlans( - parsePlan("SHOW NAMESPACES IN testcat.ns1.ns2"), - ShowNamespaces(UnresolvedNamespace(Seq("testcat", "ns1", "ns2")), None)) - comparePlans( - parsePlan("SHOW NAMESPACES IN testcat.ns1 LIKE '*pattern*'"), - ShowNamespaces(UnresolvedNamespace(Seq("testcat", "ns1")), Some("*pattern*"))) - } - test("analyze table statistics") { comparePlans(parsePlan("analyze table a.b.c compute statistics"), AnalyzeTable( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index 0d61306628a44..5c67ad9cdfe2e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -1285,95 +1285,6 @@ class DataSourceV2SQLSuite } } - test("ShowNamespaces: show root namespaces with default v2 catalog") { - spark.conf.set(SQLConf.DEFAULT_CATALOG.key, "testcat") - - testShowNamespaces("SHOW NAMESPACES", Seq()) - - spark.sql("CREATE TABLE testcat.ns1.table (id bigint) USING foo") - spark.sql("CREATE TABLE testcat.ns1.ns1_1.table (id bigint) USING foo") - spark.sql("CREATE TABLE testcat.ns2.table (id bigint) USING foo") - - testShowNamespaces("SHOW NAMESPACES", Seq("ns1", "ns2")) - testShowNamespaces("SHOW NAMESPACES LIKE '*1*'", Seq("ns1")) - } - - test("ShowNamespaces: show namespaces with v2 catalog") { - spark.sql("CREATE TABLE testcat.ns1.table (id bigint) USING foo") - spark.sql("CREATE TABLE testcat.ns1.ns1_1.table (id bigint) USING foo") - spark.sql("CREATE TABLE testcat.ns1.ns1_2.table (id bigint) USING foo") - spark.sql("CREATE TABLE testcat.ns2.table (id bigint) USING foo") - spark.sql("CREATE TABLE testcat.ns2.ns2_1.table (id bigint) USING foo") - - // Look up only with catalog name, which should list root namespaces. - testShowNamespaces("SHOW NAMESPACES IN testcat", Seq("ns1", "ns2")) - - // Look up sub-namespaces. - testShowNamespaces("SHOW NAMESPACES IN testcat.ns1", Seq("ns1.ns1_1", "ns1.ns1_2")) - testShowNamespaces("SHOW NAMESPACES IN testcat.ns1 LIKE '*2*'", Seq("ns1.ns1_2")) - testShowNamespaces("SHOW NAMESPACES IN testcat.ns2", Seq("ns2.ns2_1")) - - // Try to look up namespaces that do not exist. - testShowNamespaces("SHOW NAMESPACES IN testcat.ns3", Seq()) - testShowNamespaces("SHOW NAMESPACES IN testcat.ns1.ns3", Seq()) - } - - test("ShowNamespaces: default v2 catalog is not set") { - spark.sql("CREATE TABLE testcat.ns.table (id bigint) USING foo") - - // The current catalog is resolved to a v2 session catalog. - testShowNamespaces("SHOW NAMESPACES", Seq("default")) - } - - test("ShowNamespaces: default v2 catalog doesn't support namespace") { - spark.conf.set( - "spark.sql.catalog.testcat_no_namespace", - classOf[BasicInMemoryTableCatalog].getName) - spark.conf.set(SQLConf.DEFAULT_CATALOG.key, "testcat_no_namespace") - - val exception = intercept[AnalysisException] { - sql("SHOW NAMESPACES") - } - - assert(exception.getMessage.contains("does not support namespaces")) - } - - test("ShowNamespaces: v2 catalog doesn't support namespace") { - spark.conf.set( - "spark.sql.catalog.testcat_no_namespace", - classOf[BasicInMemoryTableCatalog].getName) - - val exception = intercept[AnalysisException] { - sql("SHOW NAMESPACES in testcat_no_namespace") - } - - assert(exception.getMessage.contains("does not support namespaces")) - } - - test("ShowNamespaces: session catalog is used and namespace doesn't exist") { - val exception = intercept[AnalysisException] { - sql("SHOW NAMESPACES in dummy") - } - - assert(exception.getMessage.contains("Namespace 'dummy' not found")) - } - - test("ShowNamespaces: change catalog and namespace with USE statements") { - sql("CREATE TABLE testcat.ns1.ns2.table (id bigint) USING foo") - - // Initially, the current catalog is a v2 session catalog. - testShowNamespaces("SHOW NAMESPACES", Seq("default")) - - // Update the current catalog to 'testcat'. - sql("USE testcat") - testShowNamespaces("SHOW NAMESPACES", Seq("ns1")) - - // Update the current namespace to 'ns1'. - sql("USE ns1") - // 'SHOW NAMESPACES' is not affected by the current namespace and lists root namespaces. - testShowNamespaces("SHOW NAMESPACES", Seq("ns1")) - } - private def testShowNamespaces( sqlText: String, expected: Seq[String]): Unit = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala index 4e2b67e532933..946e8412cfa7a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala @@ -1277,36 +1277,6 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { assertUnsupported("ALTER VIEW dbx.tab1 DROP IF EXISTS PARTITION (b='2')") } - - test("show databases") { - sql("CREATE DATABASE showdb2B") - sql("CREATE DATABASE showdb1A") - - // check the result as well as its order - checkDataset(sql("SHOW DATABASES"), Row("default"), Row("showdb1a"), Row("showdb2b")) - - checkAnswer( - sql("SHOW DATABASES LIKE '*db1A'"), - Row("showdb1a") :: Nil) - - checkAnswer( - sql("SHOW DATABASES '*db1A'"), - Row("showdb1a") :: Nil) - - checkAnswer( - sql("SHOW DATABASES LIKE 'showdb1A'"), - Row("showdb1a") :: Nil) - - checkAnswer( - sql("SHOW DATABASES LIKE '*db1A|*db2B'"), - Row("showdb1a") :: - Row("showdb2b") :: Nil) - - checkAnswer( - sql("SHOW DATABASES LIKE 'non-existentdb'"), - Nil) - } - test("drop view - temporary view") { val catalog = spark.sessionState.catalog sql( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowNamespacesParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowNamespacesParserSuite.scala new file mode 100644 index 0000000000000..c9e5d33fea87a --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowNamespacesParserSuite.scala @@ -0,0 +1,70 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.command + +import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, UnresolvedNamespace} +import org.apache.spark.sql.catalyst.parser.CatalystSqlParser.parsePlan +import org.apache.spark.sql.catalyst.parser.ParseException +import org.apache.spark.sql.catalyst.plans.logical.ShowNamespaces +import org.apache.spark.sql.test.SharedSparkSession + +class ShowNamespacesParserSuite extends AnalysisTest with SharedSparkSession { + test("all namespaces") { + Seq("SHOW NAMESPACES", "SHOW DATABASES").foreach { sqlCmd => + comparePlans( + parsePlan(sqlCmd), + ShowNamespaces(UnresolvedNamespace(Seq.empty[String]), None)) + } + } + + test("basic pattern") { + Seq( + "SHOW DATABASES LIKE 'defau*'", + "SHOW NAMESPACES LIKE 'defau*'").foreach { sqlCmd => + comparePlans( + parsePlan(sqlCmd), + ShowNamespaces(UnresolvedNamespace(Seq.empty[String]), Some("defau*"))) + } + } + + test("FROM/IN operator is not allowed by SHOW DATABASES") { + Seq( + "SHOW DATABASES FROM testcat.ns1.ns2", + "SHOW DATABASES IN testcat.ns1.ns2").foreach { sqlCmd => + val errMsg = intercept[ParseException] { + parsePlan(sqlCmd) + }.getMessage + assert(errMsg.contains("FROM/IN operator is not allowed in SHOW DATABASES")) + } + } + + test("show namespaces in/from a namespace") { + comparePlans( + parsePlan("SHOW NAMESPACES FROM testcat.ns1.ns2"), + ShowNamespaces(UnresolvedNamespace(Seq("testcat", "ns1", "ns2")), None)) + comparePlans( + parsePlan("SHOW NAMESPACES IN testcat.ns1.ns2"), + ShowNamespaces(UnresolvedNamespace(Seq("testcat", "ns1", "ns2")), None)) + } + + test("namespaces by a pattern from another namespace") { + comparePlans( + parsePlan("SHOW NAMESPACES IN testcat.ns1 LIKE '*pattern*'"), + ShowNamespaces(UnresolvedNamespace(Seq("testcat", "ns1")), Some("*pattern*"))) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowNamespacesSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowNamespacesSuiteBase.scala new file mode 100644 index 0000000000000..790489e0d47ce --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowNamespacesSuiteBase.scala @@ -0,0 +1,131 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.command + +import org.apache.spark.sql.{QueryTest, Row} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.{StringType, StructType} + +/** + * This base suite contains unified tests for the `SHOW NAMESPACES` and `SHOW DATABASES` commands + * that check V1 and V2 table catalogs. The tests that cannot run for all supported catalogs are + * located in more specific test suites: + * + * - V2 table catalog tests: `org.apache.spark.sql.execution.command.v2.ShowNamespacesSuite` + * - V1 table catalog tests: `org.apache.spark.sql.execution.command.v1.ShowNamespacesSuiteBase` + * - V1 In-Memory catalog: `org.apache.spark.sql.execution.command.v1.ShowNamespacesSuite` + * - V1 Hive External catalog: `org.apache.spark.sql.hive.execution.command.ShowNamespacesSuite` + */ +trait ShowNamespacesSuiteBase extends QueryTest with DDLCommandTestUtils { + override val command = "SHOW NAMESPACES" + + protected def runShowNamespacesSql(sqlText: String, expected: Seq[String]): Unit = { + val df = spark.sql(sqlText) + assert(df.schema === new StructType().add("namespace", StringType, false)) + checkAnswer(df, expected.map(Row(_))) + } + + protected def builtinTopNamespaces: Seq[String] = Seq.empty + + test("default namespace") { + withSQLConf(SQLConf.DEFAULT_CATALOG.key -> catalog) { + runShowNamespacesSql("SHOW NAMESPACES", builtinTopNamespaces) + } + runShowNamespacesSql(s"SHOW NAMESPACES IN $catalog", builtinTopNamespaces) + } + + test("at the top level") { + withNamespace(s"$catalog.ns1", s"$catalog.ns2") { + sql(s"CREATE DATABASE $catalog.ns1") + sql(s"CREATE NAMESPACE $catalog.ns2") + + runShowNamespacesSql( + s"SHOW NAMESPACES IN $catalog", + Seq("ns1", "ns2") ++ builtinTopNamespaces) + } + } + + test("exact matching") { + withNamespace(s"$catalog.ns1", s"$catalog.ns2") { + sql(s"CREATE NAMESPACE $catalog.ns1") + sql(s"CREATE NAMESPACE $catalog.ns2") + Seq( + s"SHOW NAMESPACES IN $catalog LIKE 'ns2'", + s"SHOW NAMESPACES IN $catalog 'ns2'", + s"SHOW NAMESPACES FROM $catalog LIKE 'ns2'", + s"SHOW NAMESPACES FROM $catalog 'ns2'").foreach { sqlCmd => + withClue(sqlCmd) { + runShowNamespacesSql(sqlCmd, Seq("ns2")) + } + } + } + } + + test("does not match to any namespace") { + Seq( + "SHOW DATABASES LIKE 'non-existentdb'", + "SHOW NAMESPACES 'non-existentdb'").foreach { sqlCmd => + runShowNamespacesSql(sqlCmd, Seq.empty) + } + } + + test("show root namespaces with the default catalog") { + withSQLConf(SQLConf.DEFAULT_CATALOG.key -> catalog) { + runShowNamespacesSql("SHOW NAMESPACES", builtinTopNamespaces) + + withNamespace("ns1", "ns2") { + sql(s"CREATE NAMESPACE ns1") + sql(s"CREATE NAMESPACE ns2") + + runShowNamespacesSql("SHOW NAMESPACES", Seq("ns1", "ns2") ++ builtinTopNamespaces) + runShowNamespacesSql("SHOW NAMESPACES LIKE '*1*'", Seq("ns1")) + } + } + } + + test("complex namespace patterns") { + withNamespace(s"$catalog.showdb2b", s"$catalog.showdb1a") { + sql(s"CREATE NAMESPACE $catalog.showdb2b") + sql(s"CREATE NAMESPACE $catalog.showdb1a") + + Seq( + "'*db1A'" -> Seq("showdb1a"), + "'*2*'" -> Seq("showdb2b"), + "'*db1A|*db2B'" -> Seq("showdb1a", "showdb2b") + ).foreach { case (pattern, expected) => + runShowNamespacesSql(s"SHOW NAMESPACES IN $catalog LIKE $pattern", expected) + } + } + } + + test("change catalog and namespace with USE statements") { + try { + withNamespace(s"$catalog.ns") { + sql(s"CREATE NAMESPACE $catalog.ns") + sql(s"USE $catalog") + runShowNamespacesSql("SHOW NAMESPACES", Seq("ns") ++ builtinTopNamespaces) + + sql("USE ns") + // 'SHOW NAMESPACES' is not affected by the current namespace and lists root namespaces. + runShowNamespacesSql("SHOW NAMESPACES", Seq("ns") ++ builtinTopNamespaces) + } + } finally { + spark.sessionState.catalogManager.reset() + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowNamespacesSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowNamespacesSuite.scala new file mode 100644 index 0000000000000..fd76ef2490f35 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowNamespacesSuite.scala @@ -0,0 +1,60 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.command.v1 + +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.execution.command +import org.apache.spark.sql.internal.SQLConf + +/** + * This base suite contains unified tests for the `SHOW NAMESPACES` and `SHOW DATABASES` commands + * that check V1 table catalogs. The tests that cannot run for all V1 catalogs are located in more + * specific test suites: + * + * - V1 In-Memory catalog: `org.apache.spark.sql.execution.command.v1.ShowNamespacesSuite` + * - V1 Hive External catalog: `org.apache.spark.sql.hive.execution.command.ShowNamespacesSuite` + */ +trait ShowNamespacesSuiteBase extends command.ShowNamespacesSuiteBase { + override protected def builtinTopNamespaces: Seq[String] = Seq("default") + + test("IN namespace doesn't exist") { + val errMsg = intercept[AnalysisException] { + sql("SHOW NAMESPACES in dummy") + }.getMessage + assert(errMsg.contains("Namespace 'dummy' not found")) + } +} + +class ShowNamespacesSuite extends ShowNamespacesSuiteBase with CommandSuiteBase { + test("case sensitivity") { + Seq(true, false).foreach { caseSensitive => + withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive.toString) { + withNamespace(s"$catalog.AAA", s"$catalog.bbb") { + sql(s"CREATE NAMESPACE $catalog.AAA") + sql(s"CREATE NAMESPACE $catalog.bbb") + val expected = if (caseSensitive) "AAA" else "aaa" + runShowNamespacesSql( + s"SHOW NAMESPACES IN $catalog", + Seq(expected, "bbb") ++ builtinTopNamespaces) + runShowNamespacesSql(s"SHOW NAMESPACES IN $catalog LIKE 'AAA'", Seq(expected)) + runShowNamespacesSql(s"SHOW NAMESPACES IN $catalog LIKE 'aaa'", Seq(expected)) + } + } + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowNamespacesSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowNamespacesSuite.scala new file mode 100644 index 0000000000000..7a2c136eeada4 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowNamespacesSuite.scala @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.command.v2 + +import org.apache.spark.SparkConf +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.connector.BasicInMemoryTableCatalog +import org.apache.spark.sql.execution.command +import org.apache.spark.sql.internal.SQLConf + +/** + * The class contains tests for the `SHOW NAMESPACES` command to check V2 table catalogs. + */ +class ShowNamespacesSuite extends command.ShowNamespacesSuiteBase with CommandSuiteBase { + override def sparkConf: SparkConf = super.sparkConf + .set("spark.sql.catalog.testcat_no_namespace", classOf[BasicInMemoryTableCatalog].getName) + + test("IN namespace doesn't exist") { + withSQLConf(SQLConf.DEFAULT_CATALOG.key -> catalog) { + runShowNamespacesSql("SHOW NAMESPACES in dummy", Seq.empty) + } + runShowNamespacesSql(s"SHOW NAMESPACES in $catalog.ns1", Seq.empty) + runShowNamespacesSql(s"SHOW NAMESPACES in $catalog.ns1.ns3", Seq.empty) + } + + test("default v2 catalog doesn't support namespace") { + withSQLConf(SQLConf.DEFAULT_CATALOG.key -> "testcat_no_namespace") { + val errMsg = intercept[AnalysisException] { + sql("SHOW NAMESPACES") + }.getMessage + assert(errMsg.contains("does not support namespaces")) + } + } + + test("v2 catalog doesn't support namespace") { + val errMsg = intercept[AnalysisException] { + sql("SHOW NAMESPACES in testcat_no_namespace") + }.getMessage + assert(errMsg.contains("does not support namespaces")) + } + + test("case sensitivity") { + Seq(true, false).foreach { caseSensitive => + withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive.toString) { + withNamespace(s"$catalog.AAA", s"$catalog.bbb") { + sql(s"CREATE NAMESPACE $catalog.AAA") + sql(s"CREATE NAMESPACE $catalog.bbb") + runShowNamespacesSql( + s"SHOW NAMESPACES IN $catalog", + Seq("AAA", "bbb") ++ builtinTopNamespaces) + runShowNamespacesSql(s"SHOW NAMESPACES IN $catalog LIKE 'AAA'", Seq("AAA")) + runShowNamespacesSql(s"SHOW NAMESPACES IN $catalog LIKE 'aaa'", Seq("AAA")) + } + } + } + } +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/ShowNamespacesSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/ShowNamespacesSuite.scala new file mode 100644 index 0000000000000..eba2569c07736 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/ShowNamespacesSuite.scala @@ -0,0 +1,43 @@ +/* + * 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.hive.execution.command + +import org.apache.spark.sql.execution.command.v1 +import org.apache.spark.sql.internal.SQLConf + +/** + * The class contains tests for the `SHOW NAMESPACES` and `SHOW DATABASES` commands to check + * V1 Hive external table catalog. + */ +class ShowNamespacesSuite extends v1.ShowNamespacesSuiteBase with CommandSuiteBase { + test("case sensitivity") { + Seq(true, false).foreach { caseSensitive => + withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive.toString) { + withNamespace(s"$catalog.AAA", s"$catalog.bbb") { + sql(s"CREATE NAMESPACE $catalog.AAA") + sql(s"CREATE NAMESPACE $catalog.bbb") + runShowNamespacesSql( + s"SHOW NAMESPACES IN $catalog", + Seq("aaa", "bbb") ++ builtinTopNamespaces) + runShowNamespacesSql(s"SHOW NAMESPACES IN $catalog LIKE 'AAA'", Seq("aaa")) + runShowNamespacesSql(s"SHOW NAMESPACES IN $catalog LIKE 'aaa'", Seq("aaa")) + } + } + } + } +} From 356fdc9a7fc88fd07751c40b920043eaebeb0abf Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Tue, 5 Jan 2021 17:20:08 +0900 Subject: [PATCH 04/29] [SPARK-34007][BUILD] Downgrade scala-maven-plugin to 4.3.0 ### What changes were proposed in this pull request? This PR is a partial revert of https://github.com/apache/spark/pull/30456 by downgrading scala-maven-plugin from 4.4.0 to 4.3.0. Currently, when you run the docker release script (`./dev/create-release/do-release-docker.sh`), it fails to compile as below during incremental compilation with zinc for an unknown reason: ``` [INFO] Compiling 21 Scala sources and 3 Java sources to /opt/spark-rm/output/spark-3.1.0-bin-hadoop2.7/resource-managers/yarn/target/scala-2.12/test-classes ... [ERROR] ## Exception when compiling 24 sources to /opt/spark-rm/output/spark-3.1.0-bin-hadoop2.7/resource-managers/yarn/target/scala-2.12/test-classes java.lang.SecurityException: class "javax.servlet.SessionCookieConfig"'s signer information does not match signer information of other classes in the same package java.lang.ClassLoader.checkCerts(ClassLoader.java:891) java.lang.ClassLoader.preDefineClass(ClassLoader.java:661) java.lang.ClassLoader.defineClass(ClassLoader.java:754) java.security.SecureClassLoader.defineClass(SecureClassLoader.java:142) java.net.URLClassLoader.defineClass(URLClassLoader.java:468) java.net.URLClassLoader.access$100(URLClassLoader.java:74) java.net.URLClassLoader$1.run(URLClassLoader.java:369) java.net.URLClassLoader$1.run(URLClassLoader.java:363) java.security.AccessController.doPrivileged(Native Method) java.net.URLClassLoader.findClass(URLClassLoader.java:362) java.lang.ClassLoader.loadClass(ClassLoader.java:418) java.lang.ClassLoader.loadClass(ClassLoader.java:351) java.lang.Class.getDeclaredMethods0(Native Method) java.lang.Class.privateGetDeclaredMethods(Class.java:2701) java.lang.Class.privateGetPublicMethods(Class.java:2902) java.lang.Class.getMethods(Class.java:1615) sbt.internal.inc.ClassToAPI$.toDefinitions0(ClassToAPI.scala:170) sbt.internal.inc.ClassToAPI$.$anonfun$toDefinitions$1(ClassToAPI.scala:123) scala.collection.mutable.HashMap.getOrElseUpdate(HashMap.scala:86) sbt.internal.inc.ClassToAPI$.toDefinitions(ClassToAPI.scala:123) sbt.internal.inc.ClassToAPI$.$anonfun$process$1(ClassToAPI.scala:3 ``` This happens when it builds Spark with Hadoop 2. It doesn't reproduce when you build this alone. It should follow the sequence of build in the release script. This is fixed by downgrading. Looks like there is a regression in scala-maven-plugin somewhere between 4.4.0 and 4.3.0. ### Why are the changes needed? To unblock the release. ### Does this PR introduce _any_ user-facing change? No, dev-only. ### How was this patch tested? It can be tested as below: ```bash ./dev/create-release/do-release-docker.sh -d $WORKING_DIR ``` Closes #31031 from HyukjinKwon/SPARK-34007. Authored-by: HyukjinKwon Signed-off-by: HyukjinKwon --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 91ca0398a076e..1282d3cd8fd2b 100644 --- a/pom.xml +++ b/pom.xml @@ -2468,7 +2468,7 @@ net.alchim31.maven scala-maven-plugin - 4.4.0 + 4.3.0 eclipse-add-source From 329850c667305053e4433c4c6da0e47b231302d4 Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Tue, 5 Jan 2021 17:21:32 +0900 Subject: [PATCH 05/29] [SPARK-32017][PYTHON][FOLLOW-UP] Rename HADOOP_VERSION to PYSPARK_HADOOP_VERSION in pip installation option ### What changes were proposed in this pull request? This PR is a followup of https://github.com/apache/spark/pull/29703. It renames `HADOOP_VERSION` environment variable to `PYSPARK_HADOOP_VERSION` in case `HADOOP_VERSION` is already being used somewhere. Arguably `HADOOP_VERSION` is a pretty common name. I see here and there: - https://www.ibm.com/support/knowledgecenter/SSZUMP_7.2.1/install_grid_sym/understanding_advanced_edition.html - https://cwiki.apache.org/confluence/display/ARROW/HDFS+Filesystem+Support - http://crs4.github.io/pydoop/_pydoop1/installation.html ### Why are the changes needed? To avoid the environment variables is unexpectedly conflicted. ### Does this PR introduce _any_ user-facing change? It renames the environment variable but it's not released yet. ### How was this patch tested? Existing unittests will test. Closes #31028 from HyukjinKwon/SPARK-32017-followup. Authored-by: HyukjinKwon Signed-off-by: HyukjinKwon --- python/docs/source/getting_started/install.rst | 10 +++++----- python/pyspark/find_spark_home.py | 2 +- python/setup.py | 14 +++++++------- 3 files changed, 13 insertions(+), 13 deletions(-) diff --git a/python/docs/source/getting_started/install.rst b/python/docs/source/getting_started/install.rst index a90f5fe159553..c5485424da664 100644 --- a/python/docs/source/getting_started/install.rst +++ b/python/docs/source/getting_started/install.rst @@ -48,11 +48,11 @@ If you want to install extra dependencies for a specific component, you can inst pip install pyspark[sql] -For PySpark with/without a specific Hadoop version, you can install it by using ``HADOOP_VERSION`` environment variables as below: +For PySpark with/without a specific Hadoop version, you can install it by using ``PYSPARK_HADOOP_VERSION`` environment variables as below: .. code-block:: bash - HADOOP_VERSION=2.7 pip install pyspark + PYSPARK_HADOOP_VERSION=2.7 pip install pyspark The default distribution uses Hadoop 3.2 and Hive 2.3. If users specify different versions of Hadoop, the pip installation automatically downloads a different version and use it in PySpark. Downloading it can take a while depending on @@ -60,15 +60,15 @@ the network and the mirror chosen. ``PYSPARK_RELEASE_MIRROR`` can be set to manu .. code-block:: bash - PYSPARK_RELEASE_MIRROR=http://mirror.apache-kr.org HADOOP_VERSION=2.7 pip install + PYSPARK_RELEASE_MIRROR=http://mirror.apache-kr.org PYSPARK_HADOOP_VERSION=2.7 pip install It is recommended to use ``-v`` option in ``pip`` to track the installation and download status. .. code-block:: bash - HADOOP_VERSION=2.7 pip install pyspark -v + PYSPARK_HADOOP_VERSION=2.7 pip install pyspark -v -Supported values in ``HADOOP_VERSION`` are: +Supported values in ``PYSPARK_HADOOP_VERSION`` are: - ``without``: Spark pre-built with user-provided Apache Hadoop - ``2.7``: Spark pre-built for Apache Hadoop 2.7 diff --git a/python/pyspark/find_spark_home.py b/python/pyspark/find_spark_home.py index 4521a36503a16..62a36d42ebc72 100755 --- a/python/pyspark/find_spark_home.py +++ b/python/pyspark/find_spark_home.py @@ -36,7 +36,7 @@ def is_spark_home(path): (os.path.isdir(os.path.join(path, "jars")) or os.path.isdir(os.path.join(path, "assembly")))) - # Spark distribution can be downloaded when HADOOP_VERSION environment variable is set. + # Spark distribution can be downloaded when PYSPARK_HADOOP_VERSION environment variable is set. # We should look up this directory first, see also SPARK-32017. spark_dist_dir = "spark-distribution" paths = [ diff --git a/python/setup.py b/python/setup.py index 7bb8a00171d37..c7f195b89aa7a 100755 --- a/python/setup.py +++ b/python/setup.py @@ -125,16 +125,16 @@ def run(self): spark_dist = os.path.join(self.install_lib, "pyspark", "spark-distribution") rmtree(spark_dist, ignore_errors=True) - if ("HADOOP_VERSION" in os.environ) or ("HIVE_VERSION" in os.environ): - # Note that SPARK_VERSION environment is just a testing purpose. - # HIVE_VERSION environment variable is also internal for now in case + if ("PYSPARK_HADOOP_VERSION" in os.environ) or ("PYSPARK_HIVE_VERSION" in os.environ): + # Note that PYSPARK_VERSION environment is just a testing purpose. + # PYSPARK_HIVE_VERSION environment variable is also internal for now in case # we support another version of Hive in the future. spark_version, hadoop_version, hive_version = install_module.checked_versions( - os.environ.get("SPARK_VERSION", VERSION).lower(), - os.environ.get("HADOOP_VERSION", install_module.DEFAULT_HADOOP).lower(), - os.environ.get("HIVE_VERSION", install_module.DEFAULT_HIVE).lower()) + os.environ.get("PYSPARK_VERSION", VERSION).lower(), + os.environ.get("PYSPARK_HADOOP_VERSION", install_module.DEFAULT_HADOOP).lower(), + os.environ.get("PYSPARK_HIVE_VERSION", install_module.DEFAULT_HIVE).lower()) - if ("SPARK_VERSION" not in os.environ and + if ("PYSPARK_VERSION" not in os.environ and ((install_module.DEFAULT_HADOOP, install_module.DEFAULT_HIVE) == (hadoop_version, hive_version))): # Do not download and install if they are same as default. From acf0a4fac2983a89c663d1622bf03a2e5929d121 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Tue, 5 Jan 2021 19:03:28 +0900 Subject: [PATCH 06/29] [SPARK-33999][BUILD] Make sbt unidoc success with JDK11 ### What changes were proposed in this pull request? This PR fixes an issue that `sbt unidoc` fails with JDK11. With the current master, `sbt unidoc` fails because the generated Java sources cause syntax error. As of JDK11, the default doclet seems to refuse such syntax error. Usually, it's enough to specify `--ignore-source-errors` option when `javadoc` runs to suppress the syntax error but unfortunately, we will then get an internal error. ``` [error] javadoc: error - An internal exception has occurred. [error] (java.lang.NullPointerException) [error] Please file a bug against the javadoc tool via the Java bug reporting page [error] (http://bugreport.java.com) after checking the Bug Database (http://bugs.java.com) [error] for duplicates. Include error messages and the following diagnostic in your report. Thank you. [error] java.lang.NullPointerException [error] at jdk.compiler/com.sun.tools.javac.code.Types.erasure(Types.java:2340) [error] at jdk.compiler/com.sun.tools.javac.code.Types$14.visitTypeVar(Types.java:2398) [error] at jdk.compiler/com.sun.tools.javac.code.Types$14.visitTypeVar(Types.java:2348) [error] at jdk.compiler/com.sun.tools.javac.code.Type$TypeVar.accept(Type.java:1659) [error] at jdk.compiler/com.sun.tools.javac.code.Types$DefaultTypeVisitor.visit(Types.java:4857) [error] at jdk.compiler/com.sun.tools.javac.code.Types.erasure(Types.java:2343) [error] at jdk.compiler/com.sun.tools.javac.code.Types.erasure(Types.java:2329) [error] at jdk.compiler/com.sun.tools.javac.model.JavacTypes.erasure(JavacTypes.java:134) [error] at jdk.javadoc/jdk.javadoc.internal.doclets.toolkit.util.Utils$5.visitTypeVariable(Utils.java:1069) [error] at jdk.javadoc/jdk.javadoc.internal.doclets.toolkit.util.Utils$5.visitTypeVariable(Utils.java:1048) [error] at jdk.compiler/com.sun.tools.javac.code.Type$TypeVar.accept(Type.java:1695) [error] at java.compiler11.0.9.1/javax.lang.model.util.AbstractTypeVisitor6.visit(AbstractTypeVisitor6.java:104) [error] at jdk.javadoc/jdk.javadoc.internal.doclets.toolkit.util.Utils.asTypeElement(Utils.java:1086) [error] at jdk.javadoc/jdk.javadoc.internal.doclets.formats.html.LinkInfoImpl.setContext(LinkInfoImpl.java:410) [error] at jdk.javadoc/jdk.javadoc.internal.doclets.formats.html.LinkInfoImpl.(LinkInfoImpl.java:285) [error] at jdk.javadoc/jdk.javadoc.internal.doclets.formats.html.LinkFactoryImpl.getTypeParameterLink(LinkFactoryImpl.java:184) [error] at jdk.javadoc/jdk.javadoc.internal.doclets.formats.html.LinkFactoryImpl.getTypeParameterLinks(LinkFactoryImpl.java:167) [error] at jdk.javadoc/jdk.javadoc.internal.doclets.toolkit.util.links.LinkFactory.getLink(LinkFactory.java:196) [error] at jdk.javadoc/jdk.javadoc.internal.doclets.formats.html.HtmlDocletWriter.getLink(HtmlDocletWriter.java:679) [error] at jdk.javadoc/jdk.javadoc.internal.doclets.formats.html.HtmlDocletWriter.addPreQualifiedClassLink(HtmlDocletWriter.java:814) [error] at jdk.javadoc/jdk.javadoc.internal.doclets.formats.html.HtmlDocletWriter.addPreQualifiedStrongClassLink(HtmlDocletWriter.java:839) [error] at jdk.javadoc/jdk.javadoc.internal.doclets.formats.html.AbstractTreeWriter.addPartialInfo(AbstractTreeWriter.java:185) [error] at jdk.javadoc/jdk.javadoc.internal.doclets.formats.html.AbstractTreeWriter.addLevelInfo(AbstractTreeWriter.java:92) [error] at jdk.javadoc/jdk.javadoc.internal.doclets.formats.html.AbstractTreeWriter.addLevelInfo(AbstractTreeWriter.java:94) [error] at jdk.javadoc/jdk.javadoc.internal.doclets.formats.html.AbstractTreeWriter.addTree(AbstractTreeWriter.java:129) [error] at jdk.javadoc/jdk.javadoc.internal.doclets.formats.html.AbstractTreeWriter.addTree(AbstractTreeWriter.java:112) [error] at jdk.javadoc/jdk.javadoc.internal.doclets.formats.html.PackageTreeWriter.generatePackageTreeFile(PackageTreeWriter.java:115) [error] at jdk.javadoc/jdk.javadoc.internal.doclets.formats.html.PackageTreeWriter.generate(PackageTreeWriter.java:92) [error] at jdk.javadoc/jdk.javadoc.internal.doclets.formats.html.HtmlDoclet.generatePackageFiles(HtmlDoclet.java:312) [error] at jdk.javadoc/jdk.javadoc.internal.doclets.toolkit.AbstractDoclet.startGeneration(AbstractDoclet.java:210) [error] at jdk.javadoc/jdk.javadoc.internal.doclets.toolkit.AbstractDoclet.run(AbstractDoclet.java:114) [error] at jdk.javadoc/jdk.javadoc.doclet.StandardDoclet.run(StandardDoclet.java:72) [error] at jdk.javadoc/jdk.javadoc.internal.tool.Start.parseAndExecute(Start.java:588) [error] at jdk.javadoc/jdk.javadoc.internal.tool.Start.begin(Start.java:432) [error] at jdk.javadoc/jdk.javadoc.internal.tool.Start.begin(Start.java:345) [error] at jdk.javadoc/jdk.javadoc.internal.tool.Main.execute(Main.java:63) [error] at jdk.javadoc/jdk.javadoc.internal.tool.Main.main(Main.java:52) ``` I found the internal error happens when a generated Java class is from a Scala class which is package private and generic. I also found that if we don't generate class hierarchy tree in the JavaDoc, we can suppress the internal error for JDK11 and later. ### Why are the changes needed? Make the build success with sbt and JDK11. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? I confirmed the following command successfully finish with JDK8 and JDK11. ``` $ build/sbt -Phive -Phive-thriftserver -Pyarn -Pkubernetes -Pmesos -Pspark-ganglia-lgpl -Pkinesis-asl -Phadoop-cloud clean unidoc ``` I also confirmed html files are successfully generated under `target/javaunidoc`. Closes #31023 from sarutak/fix-genjavadoc-java11. Authored-by: Kousuke Saruta Signed-off-by: HyukjinKwon --- project/SparkBuild.scala | 30 ++++++++++++++++++------------ 1 file changed, 18 insertions(+), 12 deletions(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index aa3e2cd65e185..668701be0ae98 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -958,18 +958,24 @@ object Unidoc { .map(_.filterNot(_.getCanonicalPath.contains("org/apache/hadoop"))) }, - javacOptions in (JavaUnidoc, unidoc) := Seq( - "-windowtitle", "Spark " + version.value.replaceAll("-SNAPSHOT", "") + " JavaDoc", - "-public", - "-noqualifier", "java.lang", - "-tag", """example:a:Example\:""", - "-tag", """note:a:Note\:""", - "-tag", "group:X", - "-tag", "tparam:X", - "-tag", "constructor:X", - "-tag", "todo:X", - "-tag", "groupname:X" - ), + javacOptions in (JavaUnidoc, unidoc) := { + val versionParts = System.getProperty("java.version").split("[+.\\-]+", 3) + var major = versionParts(0).toInt + if (major == 1) major = versionParts(1).toInt + + Seq( + "-windowtitle", "Spark " + version.value.replaceAll("-SNAPSHOT", "") + " JavaDoc", + "-public", + "-noqualifier", "java.lang", + "-tag", """example:a:Example\:""", + "-tag", """note:a:Note\:""", + "-tag", "group:X", + "-tag", "tparam:X", + "-tag", "constructor:X", + "-tag", "todo:X", + "-tag", "groupname:X", + ) ++ { if (major >= 9) Seq("--ignore-source-errors", "-notree") else Seq.empty } + }, // Use GitHub repository for Scaladoc source links unidocSourceBase := s"https://github.com/apache/spark/tree/v${version.value}", From 8d09f9649510bf5d812c82b04f7711b9252a7db0 Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Tue, 5 Jan 2021 19:48:10 +0900 Subject: [PATCH 07/29] [SPARK-34010][SQL][DODCS] Use python3 instead of python in SQL documentation build ### What changes were proposed in this pull request? This PR proposes to use python3 instead of python in SQL documentation build. After SPARK-29672, we use `sql/create-docs.sh` everywhere in Spark dev. We should fix it in `sql/create-docs.sh` too. This blocks release because the release container does not have `python` but only `python3`. ### Why are the changes needed? To unblock the release. ### Does this PR introduce _any_ user-facing change? No, dev-only. ### How was this patch tested? I manually ran the script Closes #31041 from HyukjinKwon/SPARK-34010. Authored-by: HyukjinKwon Signed-off-by: HyukjinKwon --- sql/create-docs.sh | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/sql/create-docs.sh b/sql/create-docs.sh index 6614c714e90c7..8721df874ee73 100755 --- a/sql/create-docs.sh +++ b/sql/create-docs.sh @@ -27,14 +27,14 @@ set -e FWDIR="$(cd "`dirname "${BASH_SOURCE[0]}"`"; pwd)" SPARK_HOME="$(cd "`dirname "${BASH_SOURCE[0]}"`"/..; pwd)" -if ! hash python 2>/dev/null; then - echo "Missing python in your path, skipping SQL documentation generation." +if ! hash python3 2>/dev/null; then + echo "Missing python3 in your path, skipping SQL documentation generation." exit 0 fi if ! hash mkdocs 2>/dev/null; then echo "Missing mkdocs in your path, trying to install mkdocs for SQL documentation generation." - pip install mkdocs + pip3 install mkdocs fi pushd "$FWDIR" > /dev/null From 14c2edae7e8e02e18a24862a6c113b02719d4785 Mon Sep 17 00:00:00 2001 From: huangtianhua Date: Tue, 5 Jan 2021 21:50:21 +0900 Subject: [PATCH 08/29] [SPARK-34009][BUILD] To activate profile 'aarch64' based on OS settings Instead of taking parameter '-Paarch64' when maven build to activate the profile based on OS settings automatically, than we can use same command to build on aarch64. ### What changes were proposed in this pull request? Activate profile 'aarch64' based on OS ### Why are the changes needed? After this change, we build spark using the same command for aarch64 as x86. ### Does this PR introduce _any_ user-facing change? No. After this change, no need to taking parameter '-Paarch64' when build, but take the parameter works also. ### How was this patch tested? ARM daily CI. Closes #31036 from huangtianhua/SPARK-34009. Authored-by: huangtianhua Signed-off-by: HyukjinKwon --- pom.xml | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/pom.xml b/pom.xml index 1282d3cd8fd2b..07c18f78e0735 100644 --- a/pom.xml +++ b/pom.xml @@ -3371,6 +3371,12 @@ org.openlabtesting.leveldbjni + + + linux + aarch64 + + From cc1d9d25fb4c2e4af912d6f9802de8f351c32deb Mon Sep 17 00:00:00 2001 From: gengjiaan Date: Tue, 5 Jan 2021 16:15:33 +0000 Subject: [PATCH 09/29] [SPARK-33542][SQL] Group exception messages in catalyst/catalog ### What changes were proposed in this pull request? This PR group exception messages in `/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog`. ### Why are the changes needed? It will largely help with standardization of error messages and its maintenance. ### Does this PR introduce _any_ user-facing change? No. Error messages remain unchanged. ### How was this patch tested? No new tests - pass all original tests to make sure it doesn't break any existing behavior. Closes #30870 from beliefer/SPARK-33542. Lead-authored-by: gengjiaan Co-authored-by: Jiaan Geng Co-authored-by: beliefer Signed-off-by: Wenchen Fan --- .../spark/sql/QueryCompilationErrors.scala | 170 +++++++++++++++++- .../spark/sql/QueryExecutionErrors.scala | 56 ++++++ .../catalog/GlobalTempViewManager.scala | 5 +- .../catalyst/catalog/InMemoryCatalog.scala | 42 ++--- .../sql/catalyst/catalog/SessionCatalog.scala | 78 ++++---- .../catalyst/catalog/functionResources.scala | 4 +- .../sql/catalyst/catalog/interface.scala | 45 ++--- 7 files changed, 295 insertions(+), 105 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/QueryCompilationErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/QueryCompilationErrors.scala index ff4c54df96f31..ed18e94f46ecc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/QueryCompilationErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/QueryCompilationErrors.scala @@ -17,8 +17,10 @@ package org.apache.spark.sql.errors +import org.apache.hadoop.fs.Path + import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} +import org.apache.spark.sql.catalyst.{FunctionIdentifier, QualifiedTableName, TableIdentifier} import org.apache.spark.sql.catalyst.analysis.{ResolvedNamespace, ResolvedView} import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Expression, GroupingID, NamedExpression, SpecifiedWindowFrame, WindowFrame, WindowFunction, WindowSpecDefinition} import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SerdeInfo} @@ -364,10 +366,16 @@ object QueryCompilationErrors { new AnalysisException(s"undefined function $name") } + def invalidFunctionArgumentsError( + name: String, expectedInfo: String, actualNumber: Int): Throwable = { + new AnalysisException(s"Invalid number of arguments for function $name. " + + s"Expected: $expectedInfo; Found: $actualNumber") + } + def invalidFunctionArgumentNumberError( validParametersCount: Seq[Int], name: String, params: Seq[Class[Expression]]): Throwable = { - val invalidArgumentsMsg = if (validParametersCount.length == 0) { - s"Invalid arguments for function $name" + if (validParametersCount.length == 0) { + new AnalysisException(s"Invalid arguments for function $name") } else { val expectedNumberOfParameters = if (validParametersCount.length == 1) { validParametersCount.head.toString @@ -375,10 +383,8 @@ object QueryCompilationErrors { validParametersCount.init.mkString("one of ", ", ", " and ") + validParametersCount.last } - s"Invalid number of arguments for function $name. " + - s"Expected: $expectedNumberOfParameters; Found: ${params.length}" + invalidFunctionArgumentsError(name, expectedNumberOfParameters, params.length) } - new AnalysisException(invalidArgumentsMsg) } def functionAcceptsOnlyOneArgumentError(name: String): Throwable = { @@ -504,4 +510,156 @@ object QueryCompilationErrors { def columnDoesNotExistError(colName: String): Throwable = { new AnalysisException(s"Column $colName does not exist") } + + def renameTempViewToExistingViewError(oldName: String, newName: String): Throwable = { + new AnalysisException( + s"rename temporary view from '$oldName' to '$newName': destination view already exists") + } + + def databaseNotEmptyError(db: String, details: String): Throwable = { + new AnalysisException(s"Database $db is not empty. One or more $details exist.") + } + + def invalidNameForTableOrDatabaseError(name: String): Throwable = { + new AnalysisException(s"`$name` is not a valid name for tables/databases. " + + "Valid names only contain alphabet characters, numbers and _.") + } + + def cannotCreateDatabaseWithSameNameAsPreservedDatabaseError(database: String): Throwable = { + new AnalysisException(s"$database is a system preserved database, " + + "you cannot create a database with this name.") + } + + def cannotDropDefaultDatabaseError(): Throwable = { + new AnalysisException("Can not drop default database") + } + + def cannotUsePreservedDatabaseAsCurrentDatabaseError(database: String): Throwable = { + new AnalysisException(s"$database is a system preserved database, you cannot use it as " + + "current database. To access global temporary views, you should use qualified name with " + + s"the GLOBAL_TEMP_DATABASE, e.g. SELECT * FROM $database.viewName.") + } + + def createExternalTableWithoutLocationError(): Throwable = { + new AnalysisException("CREATE EXTERNAL TABLE must be accompanied by LOCATION") + } + + def cannotOperateManagedTableWithExistingLocationError( + methodName: String, tableIdentifier: TableIdentifier, tableLocation: Path): Throwable = { + new AnalysisException(s"Can not $methodName the managed table('$tableIdentifier')" + + s". The associated location('${tableLocation.toString}') already exists.") + } + + def dropNonExistentColumnsNotSupportedError( + nonExistentColumnNames: Seq[String]): Throwable = { + new AnalysisException( + s""" + |Some existing schema fields (${nonExistentColumnNames.mkString("[", ",", "]")}) are + |not present in the new schema. We don't support dropping columns yet. + """.stripMargin) + } + + def cannotRetrieveTableOrViewNotInSameDatabaseError( + qualifiedTableNames: Seq[QualifiedTableName]): Throwable = { + new AnalysisException("Only the tables/views belong to the same database can be retrieved. " + + s"Querying tables/views are $qualifiedTableNames") + } + + def renameTableSourceAndDestinationMismatchError(db: String, newDb: String): Throwable = { + new AnalysisException( + s"RENAME TABLE source and destination databases do not match: '$db' != '$newDb'") + } + + def cannotRenameTempViewWithDatabaseSpecifiedError( + oldName: TableIdentifier, newName: TableIdentifier): Throwable = { + new AnalysisException(s"RENAME TEMPORARY VIEW from '$oldName' to '$newName': cannot " + + s"specify database name '${newName.database.get}' in the destination table") + } + + def cannotRenameTempViewToExistingTableError( + oldName: TableIdentifier, newName: TableIdentifier): Throwable = { + new AnalysisException(s"RENAME TEMPORARY VIEW from '$oldName' to '$newName': " + + "destination table already exists") + } + + def invalidPartitionSpecError(details: String): Throwable = { + new AnalysisException(s"Partition spec is invalid. $details") + } + + def functionAlreadyExistsError(func: FunctionIdentifier): Throwable = { + new AnalysisException(s"Function $func already exists") + } + + def cannotLoadClassWhenRegisteringFunctionError( + className: String, func: FunctionIdentifier): Throwable = { + new AnalysisException(s"Can not load class '$className' when registering " + + s"the function '$func', please make sure it is on the classpath") + } + + def v2CatalogNotSupportFunctionError( + catalog: String, namespace: Seq[String]): Throwable = { + new AnalysisException("V2 catalog does not support functions yet. " + + s"catalog: $catalog, namespace: '${namespace.quoted}'") + } + + def resourceTypeNotSupportedError(resourceType: String): Throwable = { + new AnalysisException(s"Resource Type '$resourceType' is not supported.") + } + + def tableNotSpecifyDatabaseError(identifier: TableIdentifier): Throwable = { + new AnalysisException(s"table $identifier did not specify database") + } + + def tableNotSpecifyLocationUriError(identifier: TableIdentifier): Throwable = { + new AnalysisException(s"table $identifier did not specify locationUri") + } + + def partitionNotSpecifyLocationUriError(specString: String): Throwable = { + new AnalysisException(s"Partition [$specString] did not specify locationUri") + } + + def invalidBucketNumberError(bucketingMaxBuckets: Int, numBuckets: Int): Throwable = { + new AnalysisException( + s"Number of buckets should be greater than 0 but less than or equal to " + + s"bucketing.maxBuckets (`$bucketingMaxBuckets`). Got `$numBuckets`") + } + + def corruptedTableNameContextInCatalogError(numParts: Int, index: Int): Throwable = { + new AnalysisException("Corrupted table name context in catalog: " + + s"$numParts parts expected, but part $index is missing.") + } + + def corruptedViewSQLConfigsInCatalogError(e: Exception): Throwable = { + new AnalysisException("Corrupted view SQL configs in catalog", cause = Some(e)) + } + + def corruptedViewQueryOutputColumnsInCatalogError(numCols: String, index: Int): Throwable = { + new AnalysisException("Corrupted view query output column names in catalog: " + + s"$numCols parts expected, but part $index is missing.") + } + + def corruptedViewReferredTempViewInCatalogError(e: Exception): Throwable = { + new AnalysisException("corrupted view referred temp view names in catalog", cause = Some(e)) + } + + def corruptedViewReferredTempFunctionsInCatalogError(e: Exception): Throwable = { + new AnalysisException( + "corrupted view referred temp functions names in catalog", cause = Some(e)) + } + + def columnStatisticsDeserializationNotSupportedError( + name: String, dataType: DataType): Throwable = { + new AnalysisException("Column statistics deserialization is not supported for " + + s"column $name of data type: $dataType.") + } + + def columnStatisticsSerializationNotSupportedError( + colName: String, dataType: DataType): Throwable = { + new AnalysisException("Column statistics serialization is not supported for " + + s"column $colName of data type: $dataType.") + } + + def cannotReadCorruptedTablePropertyError(key: String, details: String = ""): Throwable = { + new AnalysisException(s"Cannot read table property '$key' as it's corrupted.$details") + } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/QueryExecutionErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/QueryExecutionErrors.scala index d24e61c699241..61dcddb979a1b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/QueryExecutionErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/QueryExecutionErrors.scala @@ -17,7 +17,13 @@ package org.apache.spark.sql.errors +import java.io.IOException + +import org.apache.hadoop.fs.Path + +import org.apache.spark.SparkException import org.apache.spark.sql.catalyst.analysis.UnresolvedGenerator +import org.apache.spark.sql.catalyst.catalog.CatalogDatabase import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan} /** @@ -56,4 +62,54 @@ object QueryExecutionErrors { def cannotTerminateGeneratorError(generator: UnresolvedGenerator): Throwable = { new UnsupportedOperationException(s"Cannot terminate expression: $generator") } + + def unableToCreateDatabaseAsFailedToCreateDirectoryError( + dbDefinition: CatalogDatabase, e: IOException): Throwable = { + new SparkException(s"Unable to create database ${dbDefinition.name} as failed " + + s"to create its directory ${dbDefinition.locationUri}", e) + } + + def unableToDropDatabaseAsFailedToDeleteDirectoryError( + dbDefinition: CatalogDatabase, e: IOException): Throwable = { + new SparkException(s"Unable to drop database ${dbDefinition.name} as failed " + + s"to delete its directory ${dbDefinition.locationUri}", e) + } + + def unableToCreateTableAsFailedToCreateDirectoryError( + table: String, defaultTableLocation: Path, e: IOException): Throwable = { + new SparkException(s"Unable to create table $table as failed " + + s"to create its directory $defaultTableLocation", e) + } + + def unableToDeletePartitionPathError(partitionPath: Path, e: IOException): Throwable = { + new SparkException(s"Unable to delete partition path $partitionPath", e) + } + + def unableToDropTableAsFailedToDeleteDirectoryError( + table: String, dir: Path, e: IOException): Throwable = { + new SparkException(s"Unable to drop table $table as failed " + + s"to delete its directory $dir", e) + } + + def unableToRenameTableAsFailedToRenameDirectoryError( + oldName: String, newName: String, oldDir: Path, e: IOException): Throwable = { + new SparkException(s"Unable to rename table $oldName to $newName as failed " + + s"to rename its directory $oldDir", e) + } + + def unableToCreatePartitionPathError(partitionPath: Path, e: IOException): Throwable = { + new SparkException(s"Unable to create partition path $partitionPath", e) + } + + def unableToRenamePartitionPathError(oldPartPath: Path, e: IOException): Throwable = { + new SparkException(s"Unable to rename partition path $oldPartPath", e) + } + + def methodNotImplementedError(methodName: String): Throwable = { + new UnsupportedOperationException(s"$methodName is not implemented") + } + + def tableStatsNotSpecifiedError(): Throwable = { + new IllegalStateException("table stats must be specified.") + } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/GlobalTempViewManager.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/GlobalTempViewManager.scala index 6095ac0bc9c50..c7bd2a4cd800d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/GlobalTempViewManager.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/GlobalTempViewManager.scala @@ -21,10 +21,10 @@ import javax.annotation.concurrent.GuardedBy import scala.collection.mutable -import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.analysis.TempTableAlreadyExistsException import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.util.StringUtils +import org.apache.spark.sql.errors.QueryCompilationErrors /** @@ -92,8 +92,7 @@ class GlobalTempViewManager(val database: String) { def rename(oldName: String, newName: String): Boolean = synchronized { if (viewDefinitions.contains(oldName)) { if (viewDefinitions.contains(newName)) { - throw new AnalysisException( - s"rename temporary view from '$oldName' to '$newName': destination view already exists") + throw QueryCompilationErrors.renameTempViewToExistingViewError(oldName, newName) } val viewDefinition = viewDefinitions(oldName) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala index 31644a5ae4e35..64b4a112fe786 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala @@ -24,13 +24,13 @@ import scala.collection.mutable import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path -import org.apache.spark.{SparkConf, SparkException} -import org.apache.spark.sql.AnalysisException +import org.apache.spark.SparkConf import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils._ import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.catalyst.util.StringUtils +import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors} import org.apache.spark.sql.types.StructType /** @@ -112,8 +112,8 @@ class InMemoryCatalog( fs.mkdirs(location) } catch { case e: IOException => - throw new SparkException(s"Unable to create database ${dbDefinition.name} as failed " + - s"to create its directory ${dbDefinition.locationUri}", e) + throw QueryExecutionErrors.unableToCreateDatabaseAsFailedToCreateDirectoryError( + dbDefinition, e) } catalog.put(dbDefinition.name, new DatabaseDesc(dbDefinition)) } @@ -127,10 +127,10 @@ class InMemoryCatalog( if (!cascade) { // If cascade is false, make sure the database is empty. if (catalog(db).tables.nonEmpty) { - throw new AnalysisException(s"Database $db is not empty. One or more tables exist.") + throw QueryCompilationErrors.databaseNotEmptyError(db, "tables") } if (catalog(db).functions.nonEmpty) { - throw new AnalysisException(s"Database '$db' is not empty. One or more functions exist.") + throw QueryCompilationErrors.databaseNotEmptyError(db, "functions") } } // Remove the database. @@ -141,8 +141,8 @@ class InMemoryCatalog( fs.delete(location, true) } catch { case e: IOException => - throw new SparkException(s"Unable to drop database ${dbDefinition.name} as failed " + - s"to delete its directory ${dbDefinition.locationUri}", e) + throw QueryExecutionErrors.unableToDropDatabaseAsFailedToDeleteDirectoryError( + dbDefinition, e) } catalog.remove(db) } else { @@ -209,8 +209,8 @@ class InMemoryCatalog( fs.mkdirs(defaultTableLocation) } catch { case e: IOException => - throw new SparkException(s"Unable to create table $table as failed " + - s"to create its directory $defaultTableLocation", e) + throw QueryExecutionErrors.unableToCreateTableAsFailedToCreateDirectoryError( + table, defaultTableLocation, e) } tableDefinition.withNewStorage(locationUri = Some(defaultTableLocation.toUri)) } else { @@ -239,7 +239,7 @@ class InMemoryCatalog( fs.delete(partitionPath, true) } catch { case e: IOException => - throw new SparkException(s"Unable to delete partition path $partitionPath", e) + throw QueryExecutionErrors.unableToDeletePartitionPathError(partitionPath, e) } } assert(tableMeta.storage.locationUri.isDefined, @@ -252,8 +252,8 @@ class InMemoryCatalog( fs.delete(dir, true) } catch { case e: IOException => - throw new SparkException(s"Unable to drop table $table as failed " + - s"to delete its directory $dir", e) + throw QueryExecutionErrors.unableToDropTableAsFailedToDeleteDirectoryError( + table, dir, e) } } catalog(db).tables.remove(table) @@ -284,8 +284,8 @@ class InMemoryCatalog( fs.rename(oldDir, newDir) } catch { case e: IOException => - throw new SparkException(s"Unable to rename table $oldName to $newName as failed " + - s"to rename its directory $oldDir", e) + throw QueryExecutionErrors.unableToRenameTableAsFailedToRenameDirectoryError( + oldName, newName, oldDir, e) } oldDesc.table = oldDesc.table.withNewStorage(locationUri = Some(newDir.toUri)) } @@ -358,7 +358,7 @@ class InMemoryCatalog( loadPath: String, isOverwrite: Boolean, isSrcLocal: Boolean): Unit = { - throw new UnsupportedOperationException("loadTable is not implemented") + throw QueryExecutionErrors.methodNotImplementedError("loadTable") } override def loadPartition( @@ -369,7 +369,7 @@ class InMemoryCatalog( isOverwrite: Boolean, inheritTableSpecs: Boolean, isSrcLocal: Boolean): Unit = { - throw new UnsupportedOperationException("loadPartition is not implemented.") + throw QueryExecutionErrors.methodNotImplementedError("loadPartition") } override def loadDynamicPartitions( @@ -379,7 +379,7 @@ class InMemoryCatalog( partition: TablePartitionSpec, replace: Boolean, numDP: Int): Unit = { - throw new UnsupportedOperationException("loadDynamicPartitions is not implemented.") + throw QueryExecutionErrors.methodNotImplementedError("loadDynamicPartitions") } // -------------------------------------------------------------------------- @@ -416,7 +416,7 @@ class InMemoryCatalog( } } catch { case e: IOException => - throw new SparkException(s"Unable to create partition path $partitionPath", e) + throw QueryExecutionErrors.unableToCreatePartitionPathError(partitionPath, e) } existingParts.put( @@ -457,7 +457,7 @@ class InMemoryCatalog( fs.delete(partitionPath, true) } catch { case e: IOException => - throw new SparkException(s"Unable to delete partition path $partitionPath", e) + throw QueryExecutionErrors.unableToDeletePartitionPathError(partitionPath, e) } } existingParts.remove(p) @@ -490,7 +490,7 @@ class InMemoryCatalog( fs.rename(oldPartPath, newPartPath) } catch { case e: IOException => - throw new SparkException(s"Unable to rename partition path $oldPartPath", e) + throw QueryExecutionErrors.unableToRenamePartitionPathError(oldPartPath, e) } oldPartition.copy( spec = newSpec, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala index 9b542d6bd95ce..5f7028bf87c87 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala @@ -40,6 +40,7 @@ import org.apache.spark.sql.catalyst.parser.{CatalystSqlParser, ParserInterface} import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SubqueryAlias, View} import org.apache.spark.sql.catalyst.util.{CharVarcharUtils, StringUtils} import org.apache.spark.sql.connector.catalog.CatalogManager +import org.apache.spark.sql.errors.QueryCompilationErrors import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.StaticSQLConf.GLOBAL_TEMP_DATABASE import org.apache.spark.sql.types.StructType @@ -120,8 +121,7 @@ class SessionCatalog( */ private def validateName(name: String): Unit = { if (!validNameFormat.pattern.matcher(name).matches()) { - throw new AnalysisException(s"`$name` is not a valid name for tables/databases. " + - "Valid names only contain alphabet characters, numbers and _.") + throw QueryCompilationErrors.invalidNameForTableOrDatabaseError(name) } } @@ -216,9 +216,8 @@ class SessionCatalog( def createDatabase(dbDefinition: CatalogDatabase, ignoreIfExists: Boolean): Unit = { val dbName = formatDatabaseName(dbDefinition.name) if (dbName == globalTempViewManager.database) { - throw new AnalysisException( - s"${globalTempViewManager.database} is a system preserved database, " + - "you cannot create a database with this name.") + throw QueryCompilationErrors.cannotCreateDatabaseWithSameNameAsPreservedDatabaseError( + globalTempViewManager.database) } validateName(dbName) externalCatalog.createDatabase( @@ -238,7 +237,7 @@ class SessionCatalog( def dropDatabase(db: String, ignoreIfNotExists: Boolean, cascade: Boolean): Unit = { val dbName = formatDatabaseName(db) if (dbName == DEFAULT_DATABASE) { - throw new AnalysisException(s"Can not drop default database") + throw QueryCompilationErrors.cannotDropDefaultDatabaseError } if (cascade && databaseExists(dbName)) { listTables(dbName).foreach { t => @@ -279,11 +278,8 @@ class SessionCatalog( def setCurrentDatabase(db: String): Unit = { val dbName = formatDatabaseName(db) if (dbName == globalTempViewManager.database) { - throw new AnalysisException( - s"${globalTempViewManager.database} is a system preserved database, " + - "you cannot use it as current database. To access global temporary views, you should " + - "use qualified name with the GLOBAL_TEMP_DATABASE, e.g. SELECT * FROM " + - s"${globalTempViewManager.database}.viewName.") + throw QueryCompilationErrors.cannotUsePreservedDatabaseAsCurrentDatabaseError( + globalTempViewManager.database) } requireDbExists(dbName) synchronized { currentDb = dbName } @@ -320,7 +316,7 @@ class SessionCatalog( validateLocation: Boolean = true): Unit = { val isExternal = tableDefinition.tableType == CatalogTableType.EXTERNAL if (isExternal && tableDefinition.storage.locationUri.isEmpty) { - throw new AnalysisException(s"CREATE EXTERNAL TABLE must be accompanied by LOCATION") + throw QueryCompilationErrors.createExternalTableWithoutLocationError } val db = formatDatabaseName(tableDefinition.identifier.database.getOrElse(getCurrentDatabase)) @@ -359,8 +355,8 @@ class SessionCatalog( val fs = tableLocation.getFileSystem(hadoopConf) if (fs.exists(tableLocation) && fs.listStatus(tableLocation).nonEmpty) { - throw new AnalysisException(s"Can not create the managed table('${table.identifier}')" + - s". The associated location('${tableLocation.toString}') already exists.") + throw QueryCompilationErrors.cannotOperateManagedTableWithExistingLocationError( + "create", table.identifier, tableLocation) } } } @@ -428,11 +424,7 @@ class SessionCatalog( val nonExistentColumnNames = oldDataSchema.map(_.name).filterNot(columnNameResolved(newDataSchema, _)) if (nonExistentColumnNames.nonEmpty) { - throw new AnalysisException( - s""" - |Some existing schema fields (${nonExistentColumnNames.mkString("[", ",", "]")}) are - |not present in the new schema. We don't support dropping columns yet. - """.stripMargin) + throw QueryCompilationErrors.dropNonExistentColumnsNotSupportedError(nonExistentColumnNames) } externalCatalog.alterTableDataSchema(db, table, newDataSchema) @@ -508,10 +500,8 @@ class SessionCatalog( if (dbs.distinct.size != 1) { val tables = names.map(name => formatTableName(name.table)) val qualifiedTableNames = dbs.zip(tables).map { case (d, t) => QualifiedTableName(d, t)} - throw new AnalysisException( - s"Only the tables/views belong to the same database can be retrieved. Querying " + - s"tables/views are $qualifiedTableNames" - ) + throw QueryCompilationErrors.cannotRetrieveTableOrViewNotInSameDatabaseError( + qualifiedTableNames) } val db = formatDatabaseName(dbs.head) requireDbExists(db) @@ -722,8 +712,7 @@ class SessionCatalog( val db = formatDatabaseName(oldName.database.getOrElse(currentDb)) newName.database.map(formatDatabaseName).foreach { newDb => if (db != newDb) { - throw new AnalysisException( - s"RENAME TABLE source and destination databases do not match: '$db' != '$newDb'") + throw QueryCompilationErrors.renameTableSourceAndDestinationMismatchError(db, newDb) } } @@ -741,13 +730,12 @@ class SessionCatalog( externalCatalog.renameTable(db, oldTableName, newTableName) } else { if (newName.database.isDefined) { - throw new AnalysisException( - s"RENAME TEMPORARY VIEW from '$oldName' to '$newName': cannot specify database " + - s"name '${newName.database.get}' in the destination table") + throw QueryCompilationErrors.cannotRenameTempViewWithDatabaseSpecifiedError( + oldName, newName) } if (tempViews.contains(newTableName)) { - throw new AnalysisException(s"RENAME TEMPORARY VIEW from '$oldName' to '$newName': " + - "destination table already exists") + throw QueryCompilationErrors.cannotRenameTempViewToExistingTableError( + oldName, newName) } val table = tempViews(oldTableName) tempViews.remove(oldTableName) @@ -1192,8 +1180,8 @@ class SessionCatalog( specs.foreach { s => if (s.values.exists(_.isEmpty)) { val spec = s.map(p => p._1 + "=" + p._2).mkString("[", ", ", "]") - throw new AnalysisException( - s"Partition spec is invalid. The spec ($spec) contains an empty partition column value") + throw QueryCompilationErrors.invalidPartitionSpecError( + s"The spec ($spec) contains an empty partition column value") } } } @@ -1223,10 +1211,10 @@ class SessionCatalog( val defined = table.partitionColumnNames specs.foreach { s => if (!s.keys.forall(defined.contains)) { - throw new AnalysisException( - s"Partition spec is invalid. The spec (${s.keys.mkString(", ")}) must be contained " + - s"within the partition spec (${table.partitionColumnNames.mkString(", ")}) defined " + - s"in table '${table.identifier}'") + throw QueryCompilationErrors.invalidPartitionSpecError( + s"The spec (${s.keys.mkString(", ")}) must be contained " + + s"within the partition spec (${table.partitionColumnNames.mkString(", ")}) defined " + + s"in table '${table.identifier}'") } } } @@ -1382,8 +1370,8 @@ class SessionCatalog( // Check input argument size if (e.inputTypes.size != input.size) { - throw new AnalysisException(s"Invalid number of arguments for function $name. " + - s"Expected: ${e.inputTypes.size}; Found: ${input.size}") + throw QueryCompilationErrors.invalidFunctionArgumentsError( + name, e.inputTypes.size.toString, input.size) } e } else { @@ -1409,15 +1397,14 @@ class SessionCatalog( functionBuilder: Option[FunctionBuilder] = None): Unit = { val func = funcDefinition.identifier if (functionRegistry.functionExists(func) && !overrideIfExists) { - throw new AnalysisException(s"Function $func already exists") + throw QueryCompilationErrors.functionAlreadyExistsError(func) } val info = new ExpressionInfo(funcDefinition.className, func.database.orNull, func.funcName) val builder = functionBuilder.getOrElse { val className = funcDefinition.className if (!Utils.classIsLoadable(className)) { - throw new AnalysisException(s"Can not load class '$className' when registering " + - s"the function '$func', please make sure it is on the classpath") + throw QueryCompilationErrors.cannotLoadClassWhenRegisteringFunctionError(className, func) } makeFunctionBuilder(func.unquotedString, className) } @@ -1522,7 +1509,6 @@ class SessionCatalog( def lookupFunction( name: FunctionIdentifier, children: Seq[Expression]): Expression = synchronized { - import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ // Note: the implementation of this function is a little bit convoluted. // We probably shouldn't use a single FunctionRegistry to register all three kinds of functions // (built-in, temp, and external). @@ -1545,9 +1531,7 @@ class SessionCatalog( case Seq() => getCurrentDatabase case Seq(_, db) => db case Seq(catalog, namespace @ _*) => - throw new AnalysisException( - s"V2 catalog does not support functions yet. " + - s"catalog: ${catalog}, namespace: '${namespace.quoted}'") + throw QueryCompilationErrors.v2CatalogNotSupportFunctionError(catalog, namespace) } // If the name itself is not qualified, add the current database to it. @@ -1685,8 +1669,8 @@ class SessionCatalog( val newTableLocation = new Path(new Path(databaseLocation), formatTableName(newName.table)) val fs = newTableLocation.getFileSystem(hadoopConf) if (fs.exists(newTableLocation)) { - throw new AnalysisException(s"Can not rename the managed table('$oldName')" + - s". The associated location('$newTableLocation') already exists.") + throw QueryCompilationErrors.cannotOperateManagedTableWithExistingLocationError( + "rename", oldName, newTableLocation) } } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/functionResources.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/functionResources.scala index 67bf2d06c95dd..7ebe3d8c5f880 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/functionResources.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/functionResources.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.catalog import java.util.Locale -import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.errors.QueryCompilationErrors /** A trait that represents the type of a resourced needed by a function. */ abstract class FunctionResourceType(val resourceType: String) @@ -40,7 +40,7 @@ object FunctionResourceType { case "file" => FileResource case "archive" => ArchiveResource case other => - throw new AnalysisException(s"Resource Type '$resourceType' is not supported.") + throw QueryCompilationErrors.resourceTypeNotSupportedError(resourceType) } } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala index d25b1fe46d569..eb29b37a3c5d7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala @@ -29,7 +29,6 @@ import org.json4s.JsonAST.{JArray, JString} import org.json4s.jackson.JsonMethods._ import org.apache.spark.internal.Logging -import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.{FunctionIdentifier, InternalRow, SQLConfHelper, TableIdentifier} import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap, AttributeReference, Cast, ExprId, Literal} @@ -37,6 +36,7 @@ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.plans.logical.statsEstimation.EstimationUtils import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.connector.catalog.CatalogManager +import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.sql.util.CaseInsensitiveStringMap @@ -145,7 +145,7 @@ case class CatalogTablePartition( /** Return the partition location, assuming it is specified. */ def location: URI = storage.locationUri.getOrElse { val specString = spec.map { case (k, v) => s"$k=$v" }.mkString(", ") - throw new AnalysisException(s"Partition [$specString] did not specify locationUri") + throw QueryCompilationErrors.partitionNotSpecifyLocationUriError(specString) } /** @@ -182,9 +182,8 @@ case class BucketSpec( sortColumnNames: Seq[String]) extends SQLConfHelper { if (numBuckets <= 0 || numBuckets > conf.bucketingMaxBuckets) { - throw new AnalysisException( - s"Number of buckets should be greater than 0 but less than or equal to " + - s"bucketing.maxBuckets (`${conf.bucketingMaxBuckets}`). Got `$numBuckets`") + throw QueryCompilationErrors.invalidBucketNumberError( + conf.bucketingMaxBuckets, numBuckets) } override def toString: String = { @@ -274,12 +273,12 @@ case class CatalogTable( /** Return the database this table was specified to belong to, assuming it exists. */ def database: String = identifier.database.getOrElse { - throw new AnalysisException(s"table $identifier did not specify database") + throw QueryCompilationErrors.tableNotSpecifyDatabaseError(identifier) } /** Return the table location, assuming it is specified. */ def location: URI = storage.locationUri.getOrElse { - throw new AnalysisException(s"table $identifier did not specify locationUri") + throw QueryCompilationErrors.tableNotSpecifyLocationUriError(identifier) } /** Return the fully qualified name of this table, assuming the database was specified. */ @@ -295,8 +294,7 @@ case class CatalogTable( (0 until numParts).map { index => properties.getOrElse( s"$VIEW_CATALOG_AND_NAMESPACE_PART_PREFIX$index", - throw new AnalysisException("Corrupted table name context in catalog: " + - s"$numParts parts expected, but part $index is missing.") + throw QueryCompilationErrors.corruptedTableNameContextInCatalogError(numParts, index) ) } } else if (properties.contains(VIEW_DEFAULT_DATABASE)) { @@ -318,8 +316,7 @@ case class CatalogTable( yield (key.substring(CatalogTable.VIEW_SQL_CONFIG_PREFIX.length), value) } catch { case e: Exception => - throw new AnalysisException( - "Corrupted view SQL configs in catalog", cause = Some(e)) + throw QueryCompilationErrors.corruptedViewSQLConfigsInCatalogError(e) } } @@ -334,8 +331,7 @@ case class CatalogTable( index <- 0 until numCols.toInt } yield properties.getOrElse( s"$VIEW_QUERY_OUTPUT_COLUMN_NAME_PREFIX$index", - throw new AnalysisException("Corrupted view query output column names in catalog: " + - s"$numCols parts expected, but part $index is missing.") + throw QueryCompilationErrors.corruptedViewQueryOutputColumnsInCatalogError(numCols, index) ) } @@ -352,8 +348,7 @@ case class CatalogTable( }.getOrElse(Seq.empty) } catch { case e: Exception => - throw new AnalysisException( - "corrupted view referred temp view names in catalog", cause = Some(e)) + throw QueryCompilationErrors.corruptedViewReferredTempViewInCatalogError(e) } } @@ -368,8 +363,7 @@ case class CatalogTable( }.getOrElse(Seq.empty) } catch { case e: Exception => - throw new AnalysisException( - "corrupted view referred temp functions names in catalog", cause = Some(e)) + throw QueryCompilationErrors.corruptedViewReferredTempFunctionsInCatalogError(e) } } @@ -497,14 +491,13 @@ object CatalogTable { None } else { val numParts = props.get(s"$key.numParts") - val errorMessage = s"Cannot read table property '$key' as it's corrupted." if (numParts.isEmpty) { - throw new AnalysisException(errorMessage) + throw QueryCompilationErrors.cannotReadCorruptedTablePropertyError(key) } else { val parts = (0 until numParts.get.toInt).map { index => props.getOrElse(s"$key.part.$index", { - throw new AnalysisException( - s"$errorMessage Missing part $index, ${numParts.get} parts are expected.") + throw QueryCompilationErrors.cannotReadCorruptedTablePropertyError( + key, s"Missing part $index, $numParts parts are expected.") }) } Some(parts.mkString) @@ -657,8 +650,8 @@ object CatalogColumnStat extends Logging { // This version of Spark does not use min/max for binary/string types so we ignore it. case BinaryType | StringType => null case _ => - throw new AnalysisException("Column statistics deserialization is not supported for " + - s"column $name of data type: $dataType.") + throw QueryCompilationErrors.columnStatisticsDeserializationNotSupportedError( + name, dataType) } } @@ -674,8 +667,8 @@ object CatalogColumnStat extends Logging { case _: DecimalType => v.asInstanceOf[Decimal].toJavaBigDecimal // This version of Spark does not use min/max for binary/string types so we ignore it. case _ => - throw new AnalysisException("Column statistics serialization is not supported for " + - s"column $colName of data type: $dataType.") + throw QueryCompilationErrors.columnStatisticsSerializationNotSupportedError( + colName, dataType) } externalValue.toString } @@ -805,7 +798,7 @@ case class HiveTableRelation( tableMeta.stats.map(_.toPlanStats(output, conf.cboEnabled || conf.planStatsEnabled)) .orElse(tableStats) .getOrElse { - throw new IllegalStateException("table stats must be specified.") + throw QueryExecutionErrors.tableStatsNotSpecifiedError } } From 171db85aa2cdacf39caeb26162569275076fd52f Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Tue, 5 Jan 2021 13:48:52 -0800 Subject: [PATCH 10/29] [SPARK-33874][K8S][FOLLOWUP] Handle long lived sidecars - clean up logging ### What changes were proposed in this pull request? Switch log level from warn to debug when the spark container is not present in the pod's container statuses. ### Why are the changes needed? There are many non-critical situations where the Spark container may not be present, and the warning log level is too high. ### Does this PR introduce _any_ user-facing change? Log message change. ### How was this patch tested? N/A Closes #31047 from holdenk/SPARK-33874-follow-up. Authored-by: Holden Karau Signed-off-by: Dongjoon Hyun --- .../spark/scheduler/cluster/k8s/ExecutorPodsSnapshot.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsSnapshot.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsSnapshot.scala index 71355c7af10fa..37aaca7e8ceeb 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsSnapshot.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsSnapshot.scala @@ -93,9 +93,10 @@ object ExecutorPodsSnapshot extends Logging { case _ => PodRunning(pod) } - // If we can't find the Spark container status, fall back to the pod status + // If we can't find the Spark container status, fall back to the pod status. This is + // expected to occur during pod startup and other situations. case _ => - logWarning(s"Unable to find container ${sparkContainerName} in pod ${pod} " + + logDebug(s"Unable to find container ${sparkContainerName} in pod ${pod} " + "defaulting to entire pod status (running).") PodRunning(pod) } From e279ed304475a6d5a9fbf739fe9ed32ef58171cb Mon Sep 17 00:00:00 2001 From: angerszhu Date: Wed, 6 Jan 2021 08:48:24 +0900 Subject: [PATCH 11/29] [SPARK-34012][SQL] Keep behavior consistent when conf `spark.sql.legacy.parser.havingWithoutGroupByAsWhere` is true with migration guide ### What changes were proposed in this pull request? In https://github.com/apache/spark/pull/22696 we support HAVING without GROUP BY means global aggregate But since we treat having as Filter before, in this way will cause a lot of analyze error, after https://github.com/apache/spark/pull/28294 we use `UnresolvedHaving` to instead `Filter` to solve such problem, but break origin logical about treat `SELECT 1 FROM range(10) HAVING true` as `SELECT 1 FROM range(10) WHERE true` . This PR fix this issue and add UT. ### Why are the changes needed? Keep consistent behavior of migration guide. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? added UT Closes #31039 from AngersZhuuuu/SPARK-25780-Follow-up. Authored-by: angerszhu Signed-off-by: Takeshi Yamamuro --- .../sql/catalyst/parser/AstBuilder.scala | 6 +- .../resources/sql-tests/inputs/group-by.sql | 10 +++ .../sql-tests/results/group-by.sql.out | 63 ++++++++++++++++++- 3 files changed, 77 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 3ea86c6ea2abf..395a9563cdc0a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -714,7 +714,11 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg val withProject = if (aggregationClause == null && havingClause != null) { if (conf.getConf(SQLConf.LEGACY_HAVING_WITHOUT_GROUP_BY_AS_WHERE)) { // If the legacy conf is set, treat HAVING without GROUP BY as WHERE. - withHavingClause(havingClause, createProject()) + val predicate = expression(havingClause.booleanExpression) match { + case p: Predicate => p + case e => Cast(e, BooleanType) + } + Filter(predicate, createProject()) } else { // According to SQL standard, HAVING without GROUP BY means global aggregate. withHavingClause(havingClause, Aggregate(Nil, namedExpressions, withFilter)) diff --git a/sql/core/src/test/resources/sql-tests/inputs/group-by.sql b/sql/core/src/test/resources/sql-tests/inputs/group-by.sql index 81e2204358bc9..6ee1014739759 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/group-by.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/group-by.sql @@ -86,6 +86,16 @@ SELECT 1 FROM range(10) HAVING MAX(id) > 0; SELECT id FROM range(10) HAVING id > 0; +SET spark.sql.legacy.parser.havingWithoutGroupByAsWhere=true; + +SELECT 1 FROM range(10) HAVING true; + +SELECT 1 FROM range(10) HAVING MAX(id) > 0; + +SELECT id FROM range(10) HAVING id > 0; + +SET spark.sql.legacy.parser.havingWithoutGroupByAsWhere=false; + -- Test data CREATE OR REPLACE TEMPORARY VIEW test_agg AS SELECT * FROM VALUES (1, true), (1, false), diff --git a/sql/core/src/test/resources/sql-tests/results/group-by.sql.out b/sql/core/src/test/resources/sql-tests/results/group-by.sql.out index 75bda87b37642..cc07cd64f3a89 100644 --- a/sql/core/src/test/resources/sql-tests/results/group-by.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/group-by.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 57 +-- Number of queries: 62 -- !query @@ -277,6 +277,67 @@ org.apache.spark.sql.AnalysisException grouping expressions sequence is empty, and '`id`' is not an aggregate function. Wrap '()' in windowing function(s) or wrap '`id`' in first() (or first_value) if you don't care which value you get. +-- !query +SET spark.sql.legacy.parser.havingWithoutGroupByAsWhere=true +-- !query schema +struct +-- !query output +spark.sql.legacy.parser.havingWithoutGroupByAsWhere true + + +-- !query +SELECT 1 FROM range(10) HAVING true +-- !query schema +struct<1:int> +-- !query output +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 + + +-- !query +SELECT 1 FROM range(10) HAVING MAX(id) > 0 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException + +Aggregate/Window/Generate expressions are not valid in where clause of the query. +Expression in where clause: [(max(`id`) > CAST(0 AS BIGINT))] +Invalid expressions: [max(`id`)] + + +-- !query +SELECT id FROM range(10) HAVING id > 0 +-- !query schema +struct +-- !query output +1 +2 +3 +4 +5 +6 +7 +8 +9 + + +-- !query +SET spark.sql.legacy.parser.havingWithoutGroupByAsWhere=false +-- !query schema +struct +-- !query output +spark.sql.legacy.parser.havingWithoutGroupByAsWhere false + + -- !query CREATE OR REPLACE TEMPORARY VIEW test_agg AS SELECT * FROM VALUES (1, true), (1, false), From b77d11dfd942ee2164dde2f5c25c6aaed65c444c Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Wed, 6 Jan 2021 11:19:44 +0900 Subject: [PATCH 12/29] [SPARK-34011][SQL] Refresh cache in `ALTER TABLE .. RENAME TO PARTITION` ### What changes were proposed in this pull request? 1. Invoke `refreshTable()` from `AlterTableRenamePartitionCommand.run()` after partitions renaming. In particular, this re-creates the cache associated with the modified table. 2. Refresh the cache associated with tables from v2 table catalogs in the `ALTER TABLE .. RENAME TO PARTITION` command. ### Why are the changes needed? This fixes the issues portrayed by the example: ```sql spark-sql> CREATE TABLE tbl1 (col0 int, part0 int) USING parquet PARTITIONED BY (part0); spark-sql> INSERT INTO tbl1 PARTITION (part0=0) SELECT 0; spark-sql> INSERT INTO tbl1 PARTITION (part0=1) SELECT 1; spark-sql> CACHE TABLE tbl1; spark-sql> SELECT * FROM tbl1; 0 0 1 1 spark-sql> ALTER TABLE tbl1 PARTITION (part0=0) RENAME TO PARTITION (part=2); spark-sql> SELECT * FROM tbl1; 0 0 1 1 ``` The last query must not return `0 2` since `0 0` was renamed by previous command. ### Does this PR introduce _any_ user-facing change? Yes. After the changes for the example above: ```sql ... spark-sql> ALTER TABLE tbl1 PARTITION (part=0) RENAME TO PARTITION (part=2); spark-sql> SELECT * FROM tbl1; 0 2 1 1 ``` ### How was this patch tested? By running the affected test suite: ``` $ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *AlterTableRenamePartitionSuite" ``` Closes #31044 from MaxGekk/rename-partition-refresh-cache. Authored-by: Max Gekk Signed-off-by: HyukjinKwon --- .../apache/spark/sql/execution/command/ddl.scala | 1 + .../v2/AlterTableRenamePartitionExec.scala | 7 +++++-- .../datasources/v2/DataSourceV2Strategy.scala | 5 +++-- .../AlterTableRenamePartitionSuiteBase.scala | 15 +++++++++++++++ 4 files changed, 24 insertions(+), 4 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala index 5e3a67927e75a..8195d02e04b8b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala @@ -536,6 +536,7 @@ case class AlterTableRenamePartitionCommand( catalog.renamePartitions( tableName, Seq(normalizedOldPartition), Seq(normalizedNewPartition)) + sparkSession.catalog.refreshTable(table.identifier.quotedString) Seq.empty[Row] } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterTableRenamePartitionExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterTableRenamePartitionExec.scala index 38b83e3ad74e7..0632bd75102fa 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterTableRenamePartitionExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterTableRenamePartitionExec.scala @@ -28,12 +28,15 @@ import org.apache.spark.sql.connector.catalog.SupportsPartitionManagement case class AlterTableRenamePartitionExec( table: SupportsPartitionManagement, from: ResolvedPartitionSpec, - to: ResolvedPartitionSpec) extends V2CommandExec { + to: ResolvedPartitionSpec, + refreshCache: () => Unit) extends V2CommandExec { override def output: Seq[Attribute] = Seq.empty override protected def run(): Seq[InternalRow] = { - table.renamePartition(from.ident, to.ident) + if (table.renamePartition(from.ident, to.ident)) { + refreshCache() + } Seq.empty } } 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 1537ebf8f305c..fa9519bf3233c 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 @@ -368,11 +368,12 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat invalidateCache(r, recacheTable = true)) :: Nil case AlterTableRenamePartition( - ResolvedTable(_, _, table: SupportsPartitionManagement, _), from, to) => + r @ ResolvedTable(_, _, table: SupportsPartitionManagement, _), from, to) => AlterTableRenamePartitionExec( table, Seq(from).asResolvedPartitionSpecs.head, - Seq(to).asResolvedPartitionSpecs.head) :: Nil + Seq(to).asResolvedPartitionSpecs.head, + invalidateCache(r, recacheTable = true)) :: Nil case AlterTableRecoverPartitions(_: ResolvedTable) => throw new AnalysisException( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableRenamePartitionSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableRenamePartitionSuiteBase.scala index 58055262d3f11..7f66e282499d4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableRenamePartitionSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableRenamePartitionSuiteBase.scala @@ -163,4 +163,19 @@ trait AlterTableRenamePartitionSuiteBase extends QueryTest with DDLCommandTestUt } } } + + test("SPARK-34011: refresh cache after partition renaming") { + withNamespaceAndTable("ns", "tbl") { t => + sql(s"CREATE TABLE $t (id int, part int) $defaultUsing PARTITIONED BY (part)") + sql(s"INSERT INTO $t PARTITION (part=0) SELECT 0") + sql(s"INSERT INTO $t PARTITION (part=1) SELECT 1") + assert(!spark.catalog.isCached(t)) + sql(s"CACHE TABLE $t") + assert(spark.catalog.isCached(t)) + QueryTest.checkAnswer(sql(s"SELECT * FROM $t"), Seq(Row(0, 0), Row(1, 1))) + sql(s"ALTER TABLE $t PARTITION (part=0) RENAME TO PARTITION (part=2)") + assert(spark.catalog.isCached(t)) + QueryTest.checkAnswer(sql(s"SELECT * FROM $t"), Seq(Row(0, 2), Row(1, 1))) + } + } } From 3d8ee492d6cd0c086988f2970bc6ea1d70a98368 Mon Sep 17 00:00:00 2001 From: "Tom.Howland" Date: Wed, 6 Jan 2021 11:40:02 +0900 Subject: [PATCH 13/29] [SPARK-34015][R] Fixing input timing in gapply ### What changes were proposed in this pull request? When sparkR is run at log level INFO, a summary of how the worker spent its time processing the partition is printed. There is a logic error where it is over-reporting the time inputting rows. In detail: the variable inputElap in a wider context is used to mark the end of reading rows, but in the part changed here it was used as a local variable for measuring the beginning of compute time in a loop over the groups in the partition. Thus, the error is not observable if there is only one group per partition, which is what you get in unit tests. For our application, here's what a log entry looks like before these changes were applied: `20/10/09 04:08:58 INFO RRunner: Times: boot = 0.013 s, init = 0.005 s, broadcast = 0.000 s, read-input = 529.471 s, compute = 492.037 s, write-output = 0.020 s, total = 1021.546 s` this indicates that we're spending more time reading rows than operating on the rows. After these changes, it looks like this: `20/12/15 06:43:29 INFO RRunner: Times: boot = 0.013 s, init = 0.010 s, broadcast = 0.000 s, read-input = 120.275 s, compute = 1680.161 s, write-output = 0.045 s, total = 1812.553 s ` ### Why are the changes needed? Metrics shouldn't mislead? ### Does this PR introduce _any_ user-facing change? Aside from no longer misleading, no ### How was this patch tested? unit tests passed. Field test results seem plausible Closes #31021 from WamBamBoozle/input_timing. Authored-by: Tom.Howland Signed-off-by: HyukjinKwon --- R/pkg/inst/worker/worker.R | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/R/pkg/inst/worker/worker.R b/R/pkg/inst/worker/worker.R index dd271f91d0084..7fc4680bad10e 100644 --- a/R/pkg/inst/worker/worker.R +++ b/R/pkg/inst/worker/worker.R @@ -196,7 +196,7 @@ if (isEmpty != 0) { outputs <- list() for (i in seq_len(length(data))) { # Timing reading input data for execution - inputElap <- elapsedSecs() + computeStart <- elapsedSecs() output <- compute(mode, partition, serializer, deserializer, keys[[i]], colNames, computeFunc, data[[i]]) computeElap <- elapsedSecs() @@ -204,17 +204,18 @@ if (isEmpty != 0) { outputs[[length(outputs) + 1L]] <- output } else { outputResult(serializer, output, outputCon) + outputComputeElapsDiff <- outputComputeElapsDiff + (elapsedSecs() - computeElap) } - outputElap <- elapsedSecs() - computeInputElapsDiff <- computeInputElapsDiff + (computeElap - inputElap) - outputComputeElapsDiff <- outputComputeElapsDiff + (outputElap - computeElap) + computeInputElapsDiff <- computeInputElapsDiff + (computeElap - computeStart) } if (serializer == "arrow") { # See https://stat.ethz.ch/pipermail/r-help/2010-September/252046.html # rbind.fill might be an alternative to make it faster if plyr is installed. + outputStart <- elapsedSecs() combined <- do.call("rbind", outputs) SparkR:::writeSerializeInArrow(outputCon, combined) + outputComputeElapsDiff <- elapsedSecs() - outputStart } } } else { From 29510821a0e3b1e09a7710ed02a0fa1caab506af Mon Sep 17 00:00:00 2001 From: Baohe Zhang Date: Tue, 5 Jan 2021 19:16:40 -0800 Subject: [PATCH 14/29] [SPARK-33029][CORE][WEBUI] Fix the UI executor page incorrectly marking the driver as excluded ### What changes were proposed in this pull request? Filter out the driver entity when updating the exclusion status of live executors(including the driver), so the UI won't be marked as excluded in the UI even if the node that hosts the driver has been marked as excluded. ### Why are the changes needed? Before this change, if we run spark with the standalone mode and with spark.blacklist.enabled=true. The driver will be marked as excluded when the host that hosts that driver has been marked as excluded. While it's incorrect because the exclude list feature will exclude executors only and the driver is still active. ![image](https://user-images.githubusercontent.com/26694233/103238740-35c05180-4911-11eb-99a2-c87c059ba0cf.png) After the fix, the driver won't be marked as excluded. ![image](https://user-images.githubusercontent.com/26694233/103238806-6f915800-4911-11eb-80d5-3c99266cfd0a.png) ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Manual test. Reopen the UI and see the driver is no longer marked as excluded. Closes #30954 from baohe-zhang/SPARK-33029. Authored-by: Baohe Zhang Signed-off-by: Dongjoon Hyun --- .../scala/org/apache/spark/status/AppStatusListener.scala | 8 +++++--- .../executor_memory_usage_expectation.json | 4 ++-- .../executor_node_excludeOnFailure_expectation.json | 4 ++-- 3 files changed, 9 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala b/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala index bf19897e51fb3..6cb013b1a7c16 100644 --- a/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala +++ b/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala @@ -366,10 +366,12 @@ private[spark] class AppStatusListener( // Implicitly exclude every available executor for the stage associated with this node Option(liveStages.get((stageId, stageAttemptId))).foreach { stage => - val executorIds = liveExecutors.values.filter(_.host == hostId).map(_.executorId).toSeq + val executorIds = liveExecutors.values.filter(exec => exec.host == hostId + && exec.executorId != SparkContext.DRIVER_IDENTIFIER).map(_.executorId).toSeq setStageExcludedStatus(stage, now, executorIds: _*) } - liveExecutors.values.filter(_.hostname == hostId).foreach { exec => + liveExecutors.values.filter(exec => exec.hostname == hostId + && exec.executorId != SparkContext.DRIVER_IDENTIFIER).foreach { exec => addExcludedStageTo(exec, stageId, now) } } @@ -416,7 +418,7 @@ private[spark] class AppStatusListener( // Implicitly (un)exclude every executor associated with the node. liveExecutors.values.foreach { exec => - if (exec.hostname == host) { + if (exec.hostname == host && exec.executorId != SparkContext.DRIVER_IDENTIFIER) { updateExecExclusionStatus(exec, excluded, now) } } diff --git a/core/src/test/resources/HistoryServerExpectations/executor_memory_usage_expectation.json b/core/src/test/resources/HistoryServerExpectations/executor_memory_usage_expectation.json index 9adda275b5609..51449340efe9f 100644 --- a/core/src/test/resources/HistoryServerExpectations/executor_memory_usage_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/executor_memory_usage_expectation.json @@ -16,7 +16,7 @@ "totalInputBytes" : 0, "totalShuffleRead" : 0, "totalShuffleWrite" : 0, - "isBlacklisted" : true, + "isBlacklisted" : false, "maxMemory" : 908381388, "addTime" : "2016-11-16T22:33:31.477GMT", "executorLogs" : { }, @@ -30,7 +30,7 @@ "attributes" : { }, "resources" : { }, "resourceProfileId" : 0, - "isExcluded" : true, + "isExcluded" : false, "excludedInStages" : [ ] }, { "id" : "3", diff --git a/core/src/test/resources/HistoryServerExpectations/executor_node_excludeOnFailure_expectation.json b/core/src/test/resources/HistoryServerExpectations/executor_node_excludeOnFailure_expectation.json index 65bd309c1025e..47a01b2596de9 100644 --- a/core/src/test/resources/HistoryServerExpectations/executor_node_excludeOnFailure_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/executor_node_excludeOnFailure_expectation.json @@ -16,7 +16,7 @@ "totalInputBytes" : 0, "totalShuffleRead" : 0, "totalShuffleWrite" : 0, - "isBlacklisted" : true, + "isBlacklisted" : false, "maxMemory" : 908381388, "addTime" : "2016-11-16T22:33:31.477GMT", "executorLogs" : { }, @@ -30,7 +30,7 @@ "attributes" : { }, "resources" : { }, "resourceProfileId" : 0, - "isExcluded" : true, + "isExcluded" : false, "excludedInStages" : [ ] }, { "id" : "3", From 2ab77d634f2e87b080786f4f39cb17e0994bc550 Mon Sep 17 00:00:00 2001 From: gengjiaan Date: Tue, 5 Jan 2021 20:45:19 -0800 Subject: [PATCH 15/29] [SPARK-34004][SQL] Change FrameLessOffsetWindowFunction as sealed abstract class ### What changes were proposed in this pull request? Change `FrameLessOffsetWindowFunction` as sealed abstract class so that simplify pattern match. ### Why are the changes needed? Simplify pattern match ### Does this PR introduce _any_ user-facing change? Yes ### How was this patch tested? Jenkins test Closes #31026 from beliefer/SPARK-30789-followup. Lead-authored-by: gengjiaan Co-authored-by: beliefer Signed-off-by: Dongjoon Hyun --- .../org/apache/spark/sql/catalyst/analysis/Analyzer.scala | 3 --- .../spark/sql/catalyst/expressions/windowExpressions.scala | 2 +- 2 files changed, 1 insertion(+), 4 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index e41d3de642d51..883ff46148ca6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -2150,9 +2150,6 @@ class Analyzer(override val catalogManager: CatalogManager) lead.copy(ignoreNulls = ignoreNulls) case lag: Lag => lag.copy(ignoreNulls = ignoreNulls) - case _ => - throw QueryCompilationErrors.functionWithUnsupportedSyntaxError( - owf.prettyName, "IGNORE NULLS") } } else { owf diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala index b167499620c0f..1934a9b190fc7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala @@ -369,7 +369,7 @@ trait OffsetWindowFunction extends WindowFunction { * within the partition. For instance: a FrameLessOffsetWindowFunction for value x with offset -2, * will get the value of x 2 rows back from the current row in the partition. */ -abstract class FrameLessOffsetWindowFunction +sealed abstract class FrameLessOffsetWindowFunction extends OffsetWindowFunction with Unevaluable with ImplicitCastInputTypes { override def children: Seq[Expression] = Seq(input, offset, default) From b1c4fc7fc71530d2d257500484f959282f5b6d44 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Tue, 5 Jan 2021 21:50:16 -0800 Subject: [PATCH 16/29] [SPARK-34008][BUILD] Upgrade derby to 10.14.2.0 ### What changes were proposed in this pull request? This PR upgrades `derby` to `10.14.2.0`. You can check the major changes from the following URLs. * 10.13.1.1 http://svn.apache.org/repos/asf/db/derby/code/tags/10.13.1.1/RELEASE-NOTES.html * 10.14.1.0 http://svn.apache.org/repos/asf/db/derby/code/tags/10.14.1.0/RELEASE-NOTES.html * 10.14.2.0 http://svn.apache.org/repos/asf/db/derby/code/tags/10.14.2.0/RELEASE-NOTES.html ### Why are the changes needed? It seems to be the final release which supports `JDK8` as the minimum required version. After `10.15.1.3`, the minimum required version is `JDK9`. https://db.apache.org/derby/ ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Existing tests. Closes #31032 from sarutak/upgrade-derby. Authored-by: Kousuke Saruta Signed-off-by: Dongjoon Hyun --- dev/deps/spark-deps-hadoop-2.7-hive-2.3 | 2 +- dev/deps/spark-deps-hadoop-3.2-hive-2.3 | 2 +- pom.xml | 3 ++- 3 files changed, 4 insertions(+), 3 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-2.7-hive-2.3 b/dev/deps/spark-deps-hadoop-2.7-hive-2.3 index 9c516203dd3fa..8d8ef2e972098 100644 --- a/dev/deps/spark-deps-hadoop-2.7-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-2.7-hive-2.3 @@ -58,7 +58,7 @@ curator-recipes/2.7.1//curator-recipes-2.7.1.jar datanucleus-api-jdo/4.2.4//datanucleus-api-jdo-4.2.4.jar datanucleus-core/4.1.17//datanucleus-core-4.1.17.jar datanucleus-rdbms/4.1.19//datanucleus-rdbms-4.1.19.jar -derby/10.12.1.1//derby-10.12.1.1.jar +derby/10.14.2.0//derby-10.14.2.0.jar dropwizard-metrics-hadoop-metrics2-reporter/0.1.2//dropwizard-metrics-hadoop-metrics2-reporter-0.1.2.jar flatbuffers-java/1.9.0//flatbuffers-java-1.9.0.jar generex/1.0.2//generex-1.0.2.jar diff --git a/dev/deps/spark-deps-hadoop-3.2-hive-2.3 b/dev/deps/spark-deps-hadoop-3.2-hive-2.3 index 1d80fadb5762a..bf56fc18c0446 100644 --- a/dev/deps/spark-deps-hadoop-3.2-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-3.2-hive-2.3 @@ -55,7 +55,7 @@ curator-recipes/2.13.0//curator-recipes-2.13.0.jar datanucleus-api-jdo/4.2.4//datanucleus-api-jdo-4.2.4.jar datanucleus-core/4.1.17//datanucleus-core-4.1.17.jar datanucleus-rdbms/4.1.19//datanucleus-rdbms-4.1.19.jar -derby/10.12.1.1//derby-10.12.1.1.jar +derby/10.14.2.0//derby-10.14.2.0.jar dnsjava/2.1.7//dnsjava-2.1.7.jar dropwizard-metrics-hadoop-metrics2-reporter/0.1.2//dropwizard-metrics-hadoop-metrics2-reporter-0.1.2.jar ehcache/3.3.1//ehcache-3.3.1.jar diff --git a/pom.xml b/pom.xml index 07c18f78e0735..f921e35a76b41 100644 --- a/pom.xml +++ b/pom.xml @@ -134,7 +134,8 @@ 2.3 2.6.0 - 10.12.1.1 + + 10.14.2.0 1.10.1 1.6.6 9.4.34.v20201102 From fa9309001a47a2b87f7a735f964537886ed9bd4c Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Tue, 5 Jan 2021 21:59:49 -0800 Subject: [PATCH 17/29] [SPARK-33635][SS] Adjust the order of check in KafkaTokenUtil.needTokenUpdate to remedy perf regression ### What changes were proposed in this pull request? This PR proposes to adjust the order of check in KafkaTokenUtil.needTokenUpdate, so that short-circuit applies on the non-delegation token cases (insecure + secured without delegation token) and remedies the performance regression heavily. ### Why are the changes needed? There's a serious performance regression between Spark 2.4 vs Spark 3.0 on read path against Kafka data source. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Manually ran a reproducer (https://github.com/codegorillauk/spark-kafka-read with modification to just count instead of writing to Kafka topic) with measuring the time. > the branch applying the change with adding measurement https://github.com/HeartSaVioR/spark/commits/debug-SPARK-33635-v3.0.1 > the branch only adding measurement https://github.com/HeartSaVioR/spark/commits/debug-original-ver-SPARK-33635-v3.0.1 > the result (before the fix) count: 10280000 Took 41.634007047 secs 21/01/06 13:16:07 INFO KafkaDataConsumer: debug ver. 17-original 21/01/06 13:16:07 INFO KafkaDataConsumer: Total time taken to retrieve: 82118 ms > the result (after the fix) count: 10280000 Took 7.964058475 secs 21/01/06 13:08:22 INFO KafkaDataConsumer: debug ver. 17 21/01/06 13:08:22 INFO KafkaDataConsumer: Total time taken to retrieve: 987 ms Closes #31056 from HeartSaVioR/SPARK-33635. Authored-by: Jungtaek Lim (HeartSaVioR) Signed-off-by: Dongjoon Hyun --- .../main/scala/org/apache/spark/kafka010/KafkaTokenUtil.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/external/kafka-0-10-token-provider/src/main/scala/org/apache/spark/kafka010/KafkaTokenUtil.scala b/external/kafka-0-10-token-provider/src/main/scala/org/apache/spark/kafka010/KafkaTokenUtil.scala index bc790418decd3..f3f6b4de6f79c 100644 --- a/external/kafka-0-10-token-provider/src/main/scala/org/apache/spark/kafka010/KafkaTokenUtil.scala +++ b/external/kafka-0-10-token-provider/src/main/scala/org/apache/spark/kafka010/KafkaTokenUtil.scala @@ -273,8 +273,8 @@ private[spark] object KafkaTokenUtil extends Logging { sparkConf: SparkConf, params: ju.Map[String, Object], clusterConfig: Option[KafkaTokenClusterConf]): Boolean = { - if (HadoopDelegationTokenManager.isServiceEnabled(sparkConf, "kafka") && - clusterConfig.isDefined && params.containsKey(SaslConfigs.SASL_JAAS_CONFIG)) { + if (clusterConfig.isDefined && params.containsKey(SaslConfigs.SASL_JAAS_CONFIG) && + HadoopDelegationTokenManager.isServiceEnabled(sparkConf, "kafka")) { logDebug("Delegation token used by connector, checking if uses the latest token.") val connectorJaasParams = params.get(SaslConfigs.SASL_JAAS_CONFIG).asInstanceOf[String] getTokenJaasParams(clusterConfig.get) != connectorJaasParams From c0d0dbabdb264180d5a88e2656e4a2fe353f21f1 Mon Sep 17 00:00:00 2001 From: angerszhu Date: Tue, 5 Jan 2021 22:33:15 -0800 Subject: [PATCH 18/29] [SPARK-33934][SQL][FOLLOW-UP] Use SubProcessor's exit code as assert condition to fix flaky test ### What changes were proposed in this pull request? Follow comment and fix. flaky test https://github.com/apache/spark/pull/30973#issuecomment-754852130. This flaky test is similar as https://github.com/apache/spark/pull/30896 Some task's failed with root cause but in driver may return error without root cause , change. UT to check with status exit code since different root cause's exit code is not same. ### Why are the changes needed? Fix flaky test ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Existed UT Closes #31046 from AngersZhuuuu/SPARK-33934-FOLLOW-UP. Lead-authored-by: angerszhu Co-authored-by: AngersZhuuuu Signed-off-by: Dongjoon Hyun --- .../spark/sql/execution/BaseScriptTransformationExec.scala | 2 +- sql/core/src/test/resources/test_script.py | 2 +- .../spark/sql/execution/BaseScriptTransformationSuite.scala | 5 ++++- 3 files changed, 6 insertions(+), 3 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/BaseScriptTransformationExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/BaseScriptTransformationExec.scala index b66f94ae1107a..669b90f4d06dd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/BaseScriptTransformationExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/BaseScriptTransformationExec.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.execution -import java.io._ +import java.io.{BufferedReader, File, InputStream, InputStreamReader, OutputStream} import java.nio.charset.StandardCharsets import java.util.concurrent.TimeUnit diff --git a/sql/core/src/test/resources/test_script.py b/sql/core/src/test/resources/test_script.py index 75b4f106d3a1a..4fcd483f44d43 100644 --- a/sql/core/src/test/resources/test_script.py +++ b/sql/core/src/test/resources/test_script.py @@ -1,4 +1,4 @@ -#! /usr/bin/python +#!/usr/bin/env python3 # Licensed to the Apache Software Foundation (ASF) under one or more # contributor license agreements. See the NOTICE file distributed with diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/BaseScriptTransformationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/BaseScriptTransformationSuite.scala index a25e4b8f8ea07..cef870b249985 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/BaseScriptTransformationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/BaseScriptTransformationSuite.scala @@ -521,7 +521,10 @@ abstract class BaseScriptTransformationSuite extends SparkPlanTest with SQLTestU 'd.cast("string"), 'e.cast("string")).collect()) }.getMessage - assert(e1.contains("Permission denied")) + // Check with status exit code since in GA test, it may lose detail failed root cause. + // Different root cause's exitcode is not same. + // In this test, root cause is `Permission denied` + assert(e1.contains("Subprocess exited with status 126")) // test `/path/to/script.py' with script executable scriptFilePath.setExecutable(true) From 45a4ff8e5472ed724b1bba40ce4ee5d314bf72c2 Mon Sep 17 00:00:00 2001 From: yangjie01 Date: Tue, 5 Jan 2021 23:11:23 -0800 Subject: [PATCH 19/29] [SPARK-33948][SQL] Fix CodeGen error of MapObjects.doGenCode method in Scala 2.13 ### What changes were proposed in this pull request? `MapObjects.doGenCode` method will generate wrong code when `inputDataType` is `ArrayBuffer`. For example `encode/decode for Tuple2: (ArrayBuffer[(String, String)],ArrayBuffer((a,b))) (codegen path)` in `ExpressionEncoderSuite`, the error generated code part as follow: ``` /* 126 */ private scala.collection.mutable.ArrayBuffer MapObjects_0(InternalRow i) { /* 127 */ boolean isNull_4 = i.isNullAt(1); /* 128 */ ArrayData value_4 = isNull_4 ? /* 129 */ null : (i.getArray(1)); /* 130 */ scala.collection.mutable.ArrayBuffer value_3 = null; /* 131 */ /* 132 */ if (!isNull_4) { /* 133 */ /* 134 */ int dataLength_0 = value_4.numElements(); /* 135 */ /* 136 */ scala.Tuple2[] convertedArray_0 = null; /* 137 */ convertedArray_0 = new scala.Tuple2[dataLength_0]; /* 138 */ /* 139 */ /* 140 */ int loopIndex_0 = 0; /* 141 */ /* 142 */ while (loopIndex_0 < dataLength_0) { /* 143 */ value_MapObject_lambda_variable_1 = (InternalRow) (value_4.getStruct(loopIndex_0, 2)); /* 144 */ isNull_MapObject_lambda_variable_1 = value_4.isNullAt(loopIndex_0); /* 145 */ /* 146 */ boolean isNull_5 = false; /* 147 */ scala.Tuple2 value_5 = null; /* 148 */ if (!false && isNull_MapObject_lambda_variable_1) { /* 149 */ /* 150 */ isNull_5 = true; /* 151 */ value_5 = ((scala.Tuple2)null); /* 152 */ } else { /* 153 */ scala.Tuple2 value_13 = NewInstance_0(i); /* 154 */ isNull_5 = false; /* 155 */ value_5 = value_13; /* 156 */ } /* 157 */ if (isNull_5) { /* 158 */ convertedArray_0[loopIndex_0] = null; /* 159 */ } else { /* 160 */ convertedArray_0[loopIndex_0] = value_5; /* 161 */ } /* 162 */ /* 163 */ loopIndex_0 += 1; /* 164 */ } /* 165 */ /* 166 */ value_3 = new org.apache.spark.sql.catalyst.util.GenericArrayData(convertedArray_0); /* 167 */ } /* 168 */ globalIsNull_0 = isNull_4; /* 169 */ return value_3; /* 170 */ } ``` Line 166 in generated code try to assign `GenericArrayData` to `value_3(ArrayBuffer)` because `ArrayBuffer` type can't match `s.c.i.Seq` branch in Scala 2.13 in `MapObjects.doGenCode` method now. So this pr change to use `s.c.Seq` instead of `Seq` alias to let `ArrayBuffer` type can enter the same branch as Scala 2.12. After this pr the generate code when `inputDataType` is `ArrayBuffer` as follow: ``` /* 126 */ private scala.collection.mutable.ArrayBuffer MapObjects_0(InternalRow i) { /* 127 */ boolean isNull_4 = i.isNullAt(1); /* 128 */ ArrayData value_4 = isNull_4 ? /* 129 */ null : (i.getArray(1)); /* 130 */ scala.collection.mutable.ArrayBuffer value_3 = null; /* 131 */ /* 132 */ if (!isNull_4) { /* 133 */ /* 134 */ int dataLength_0 = value_4.numElements(); /* 135 */ /* 136 */ scala.collection.mutable.Builder collectionBuilder_0 = scala.collection.mutable.ArrayBuffer$.MODULE$.newBuilder(); /* 137 */ collectionBuilder_0.sizeHint(dataLength_0); /* 138 */ /* 139 */ /* 140 */ int loopIndex_0 = 0; /* 141 */ /* 142 */ while (loopIndex_0 < dataLength_0) { /* 143 */ value_MapObject_lambda_variable_1 = (InternalRow) (value_4.getStruct(loopIndex_0, 2)); /* 144 */ isNull_MapObject_lambda_variable_1 = value_4.isNullAt(loopIndex_0); /* 145 */ /* 146 */ boolean isNull_5 = false; /* 147 */ scala.Tuple2 value_5 = null; /* 148 */ if (!false && isNull_MapObject_lambda_variable_1) { /* 149 */ /* 150 */ isNull_5 = true; /* 151 */ value_5 = ((scala.Tuple2)null); /* 152 */ } else { /* 153 */ scala.Tuple2 value_13 = NewInstance_0(i); /* 154 */ isNull_5 = false; /* 155 */ value_5 = value_13; /* 156 */ } /* 157 */ if (isNull_5) { /* 158 */ collectionBuilder_0.$plus$eq(null); /* 159 */ } else { /* 160 */ collectionBuilder_0.$plus$eq(value_5); /* 161 */ } /* 162 */ /* 163 */ loopIndex_0 += 1; /* 164 */ } /* 165 */ /* 166 */ value_3 = (scala.collection.mutable.ArrayBuffer) collectionBuilder_0.result(); /* 167 */ } /* 168 */ globalIsNull_0 = isNull_4; /* 169 */ return value_3; /* 170 */ } ``` ### Why are the changes needed? Bug fix in Scala 2.13 ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? - Pass the Jenkins or GitHub Action - Manual test `sql/catalyst` and `sql/core` in Scala 2.13 passed ``` mvn clean test -pl sql/catalyst -Pscala-2.13 Run completed in 11 minutes, 23 seconds. Total number of tests run: 4711 Suites: completed 261, aborted 0 Tests: succeeded 4711, failed 0, canceled 0, ignored 5, pending 0 All tests passed. ``` - Manual cherry-pick this pr to branch 3.1 and test`sql/catalyst` in Scala 2.13 passed ``` mvn clean test -pl sql/catalyst -Pscala-2.13 Run completed in 11 minutes, 18 seconds. Total number of tests run: 4655 Suites: completed 256, aborted 0 Tests: succeeded 4655, failed 0, canceled 0, ignored 5, pending 0 ``` Closes #31055 from LuciferYang/SPARK-33948. Authored-by: yangjie01 Signed-off-by: Dongjoon Hyun --- .../apache/spark/sql/catalyst/expressions/objects/objects.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala index 9303df75af503..f391b3128cf41 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala @@ -954,7 +954,7 @@ case class MapObjects private( } else { doCodeGenForScala213 } - case Some(cls) if classOf[Seq[_]].isAssignableFrom(cls) || + case Some(cls) if classOf[scala.collection.Seq[_]].isAssignableFrom(cls) || classOf[scala.collection.Set[_]].isAssignableFrom(cls) => // Scala sequence or set val getBuilder = s"${cls.getName}$$.MODULE$$.newBuilder()" From 26d8df300a1a57e220b1a0f9814795f68101f28b Mon Sep 17 00:00:00 2001 From: gengjiaan Date: Wed, 6 Jan 2021 08:25:34 +0000 Subject: [PATCH 20/29] [SPARK-33938][SQL] Optimize Like Any/All by LikeSimplification ### What changes were proposed in this pull request? We should optimize Like Any/All by LikeSimplification to improve performance. ### Why are the changes needed? Optimize Like Any/All ### Does this PR introduce _any_ user-facing change? 'No'. ### How was this patch tested? Jenkins test. Closes #30975 from beliefer/SPARK-33938. Lead-authored-by: gengjiaan Co-authored-by: beliefer Co-authored-by: Jiaan Geng Signed-off-by: Wenchen Fan --- .../expressions/regexpExpressions.scala | 6 +- .../sql/catalyst/optimizer/expressions.scala | 81 +++++++++++++------ .../optimizer/LikeSimplificationSuite.scala | 68 ++++++++++++++++ 3 files changed, 128 insertions(+), 27 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala index dae954a579eb3..011371a513a8d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala @@ -181,7 +181,7 @@ case class Like(left: Expression, right: Expression, escapeChar: Char) } } -abstract class MultiLikeBase +sealed abstract class MultiLikeBase extends UnaryExpression with ImplicitCastInputTypes with NullIntolerant { protected def patterns: Seq[UTF8String] @@ -220,7 +220,7 @@ abstract class MultiLikeBase /** * Optimized version of LIKE ALL, when all pattern values are literal. */ -abstract class LikeAllBase extends MultiLikeBase { +sealed abstract class LikeAllBase extends MultiLikeBase { override def matches(exprValue: String): Any = { if (cache.forall(matchFunc(_, exprValue))) { @@ -276,7 +276,7 @@ case class NotLikeAll(child: Expression, patterns: Seq[UTF8String]) extends Like /** * Optimized version of LIKE ANY, when all pattern values are literal. */ -abstract class LikeAnyBase extends MultiLikeBase { +sealed abstract class LikeAnyBase extends MultiLikeBase { override def matches(exprValue: String): Any = { if (cache.exists(matchFunc(_, exprValue))) { 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 819bffeafb643..a40456da82977 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 @@ -21,7 +21,7 @@ import scala.collection.immutable.HashSet import scala.collection.mutable.{ArrayBuffer, Stack} import org.apache.spark.sql.catalyst.analysis._ -import org.apache.spark.sql.catalyst.expressions.{BinaryExpression, _} +import org.apache.spark.sql.catalyst.expressions.{BinaryExpression, MultiLikeBase, _} import org.apache.spark.sql.catalyst.expressions.Literal.{FalseLiteral, TrueLiteral} import org.apache.spark.sql.catalyst.expressions.aggregate._ import org.apache.spark.sql.catalyst.expressions.objects.AssertNotNull @@ -30,6 +30,7 @@ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.types.UTF8String /* * Optimization rules defined in this file should not affect the structure of the logical plan. @@ -634,36 +635,68 @@ object LikeSimplification extends Rule[LogicalPlan] { private val contains = "%([^_%]+)%".r private val equalTo = "([^_%]*)".r + private def simplifyLike( + input: Expression, pattern: String, escapeChar: Char = '\\'): Option[Expression] = { + if (pattern.contains(escapeChar)) { + // There are three different situations when pattern containing escapeChar: + // 1. pattern contains invalid escape sequence, e.g. 'm\aca' + // 2. pattern contains escaped wildcard character, e.g. 'ma\%ca' + // 3. pattern contains escaped escape character, e.g. 'ma\\ca' + // Although there are patterns can be optimized if we handle the escape first, we just + // skip this rule if pattern contains any escapeChar for simplicity. + None + } else { + pattern match { + case startsWith(prefix) => + Some(StartsWith(input, Literal(prefix))) + case endsWith(postfix) => + Some(EndsWith(input, Literal(postfix))) + // 'a%a' pattern is basically same with 'a%' && '%a'. + // However, the additional `Length` condition is required to prevent 'a' match 'a%a'. + case startsAndEndsWith(prefix, postfix) => + Some(And(GreaterThanOrEqual(Length(input), Literal(prefix.length + postfix.length)), + And(StartsWith(input, Literal(prefix)), EndsWith(input, Literal(postfix))))) + case contains(infix) => + Some(Contains(input, Literal(infix))) + case equalTo(str) => + Some(EqualTo(input, Literal(str))) + case _ => None + } + } + } + + private def simplifyMultiLike( + child: Expression, patterns: Seq[UTF8String], multi: MultiLikeBase): Expression = { + val (remainPatternMap, replacementMap) = + patterns.map { p => p -> simplifyLike(child, p.toString)}.partition(_._2.isEmpty) + val remainPatterns = remainPatternMap.map(_._1) + val replacements = replacementMap.map(_._2.get) + if (replacements.isEmpty) { + multi + } else { + multi match { + case l: LikeAll => And(replacements.reduceLeft(And), l.copy(patterns = remainPatterns)) + case l: NotLikeAll => + And(replacements.map(Not(_)).reduceLeft(And), l.copy(patterns = remainPatterns)) + case l: LikeAny => Or(replacements.reduceLeft(Or), l.copy(patterns = remainPatterns)) + case l: NotLikeAny => + Or(replacements.map(Not(_)).reduceLeft(Or), l.copy(patterns = remainPatterns)) + } + } + } + def apply(plan: LogicalPlan): LogicalPlan = plan transformAllExpressions { case l @ Like(input, Literal(pattern, StringType), escapeChar) => if (pattern == null) { // If pattern is null, return null value directly, since "col like null" == null. Literal(null, BooleanType) } else { - pattern.toString match { - // There are three different situations when pattern containing escapeChar: - // 1. pattern contains invalid escape sequence, e.g. 'm\aca' - // 2. pattern contains escaped wildcard character, e.g. 'ma\%ca' - // 3. pattern contains escaped escape character, e.g. 'ma\\ca' - // Although there are patterns can be optimized if we handle the escape first, we just - // skip this rule if pattern contains any escapeChar for simplicity. - case p if p.contains(escapeChar) => l - case startsWith(prefix) => - StartsWith(input, Literal(prefix)) - case endsWith(postfix) => - EndsWith(input, Literal(postfix)) - // 'a%a' pattern is basically same with 'a%' && '%a'. - // However, the additional `Length` condition is required to prevent 'a' match 'a%a'. - case startsAndEndsWith(prefix, postfix) => - And(GreaterThanOrEqual(Length(input), Literal(prefix.length + postfix.length)), - And(StartsWith(input, Literal(prefix)), EndsWith(input, Literal(postfix)))) - case contains(infix) => - Contains(input, Literal(infix)) - case equalTo(str) => - EqualTo(input, Literal(str)) - case _ => l - } + simplifyLike(input, pattern.toString, escapeChar).getOrElse(l) } + case l @ LikeAll(child, patterns) => simplifyMultiLike(child, patterns, l) + case l @ NotLikeAll(child, patterns) => simplifyMultiLike(child, patterns, l) + case l @ LikeAny(child, patterns) => simplifyMultiLike(child, patterns, l) + case l @ NotLikeAny(child, patterns) => simplifyMultiLike(child, patterns, l) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LikeSimplificationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LikeSimplificationSuite.scala index 1812dce0da426..c06c92f9c1511 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LikeSimplificationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LikeSimplificationSuite.scala @@ -164,4 +164,72 @@ class LikeSimplificationSuite extends PlanTest { .analyze comparePlans(optimized5, correctAnswer5) } + + test("simplify LikeAll") { + val originalQuery = + testRelation + .where(('a likeAll( + "abc%", "abc\\%", "%xyz", "abc\\%def", "abc%def", "%mn%", "%mn\\%", "", "abc"))) + + val optimized = Optimize.execute(originalQuery.analyze) + val correctAnswer = testRelation + .where((((((StartsWith('a, "abc") && EndsWith('a, "xyz")) && + (Length('a) >= 6 && (StartsWith('a, "abc") && EndsWith('a, "def")))) && + Contains('a, "mn")) && ('a === "")) && ('a === "abc")) && + ('a likeAll("abc\\%", "abc\\%def", "%mn\\%"))) + .analyze + + comparePlans(optimized, correctAnswer) + } + + test("simplify NotLikeAll") { + val originalQuery = + testRelation + .where(('a notLikeAll( + "abc%", "abc\\%", "%xyz", "abc\\%def", "abc%def", "%mn%", "%mn\\%", "", "abc"))) + + val optimized = Optimize.execute(originalQuery.analyze) + val correctAnswer = testRelation + .where((((((Not(StartsWith('a, "abc")) && Not(EndsWith('a, "xyz"))) && + Not(Length('a) >= 6 && (StartsWith('a, "abc") && EndsWith('a, "def")))) && + Not(Contains('a, "mn"))) && Not('a === "")) && Not('a === "abc")) && + ('a notLikeAll("abc\\%", "abc\\%def", "%mn\\%"))) + .analyze + + comparePlans(optimized, correctAnswer) + } + + test("simplify LikeAny") { + val originalQuery = + testRelation + .where(('a likeAny( + "abc%", "abc\\%", "%xyz", "abc\\%def", "abc%def", "%mn%", "%mn\\%", "", "abc"))) + + val optimized = Optimize.execute(originalQuery.analyze) + val correctAnswer = testRelation + .where((((((StartsWith('a, "abc") || EndsWith('a, "xyz")) || + (Length('a) >= 6 && (StartsWith('a, "abc") && EndsWith('a, "def")))) || + Contains('a, "mn")) || ('a === "")) || ('a === "abc")) || + ('a likeAny("abc\\%", "abc\\%def", "%mn\\%"))) + .analyze + + comparePlans(optimized, correctAnswer) + } + + test("simplify NotLikeAny") { + val originalQuery = + testRelation + .where(('a notLikeAny( + "abc%", "abc\\%", "%xyz", "abc\\%def", "abc%def", "%mn%", "%mn\\%", "", "abc"))) + + val optimized = Optimize.execute(originalQuery.analyze) + val correctAnswer = testRelation + .where((((((Not(StartsWith('a, "abc")) || Not(EndsWith('a, "xyz"))) || + Not(Length('a) >= 6 && (StartsWith('a, "abc") && EndsWith('a, "def")))) || + Not(Contains('a, "mn"))) || Not('a === "")) || Not('a === "abc")) || + ('a notLikeAny("abc\\%", "abc\\%def", "%mn\\%"))) + .analyze + + comparePlans(optimized, correctAnswer) + } } From f64dfa8727b785f333a0c10f5f7175ab51f22764 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Wed, 6 Jan 2021 14:55:40 +0530 Subject: [PATCH 21/29] [SPARK-32221][K8S] Avoid possible errors due to incorrect file size or type supplied in spark conf ### What changes were proposed in this pull request? Skip files if they are binary or very large to fit the configMap's max size. ### Why are the changes needed? Config map cannot hold binary files and there is also a limit on how much data a configMap can hold. This limit can be configured by the k8s cluster admin. This PR, skips such files (with a warning) instead of failing with weird runtime errors. If such files are not skipped, then it would result in mount errors or encoding errors (if binary files are submitted). ### Does this PR introduce _any_ user-facing change? yes, in simple words avoids possible errors due to negligence (for example, placing a large file or a binary file in SPARK_CONF_DIR) and thus improves user experience. ### How was this patch tested? Added relevant tests and improved existing tests. Closes #30472 from ScrapCodes/SPARK-32221/avoid-conf-propagate-errors. Lead-authored-by: Prashant Sharma Co-authored-by: Prashant Sharma Signed-off-by: Prashant Sharma --- .../org/apache/spark/deploy/k8s/Config.scala | 8 ++ .../k8s/submit/KubernetesClientUtils.scala | 80 +++++++++++++++---- .../spark/deploy/k8s/submit/ClientSuite.scala | 21 +++-- .../submit/KubernetesClientUtilsSuite.scala | 79 ++++++++++++++++++ 4 files changed, 164 insertions(+), 24 deletions(-) create mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientUtilsSuite.scala diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala index 6939de4697979..8dca875b543c6 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala @@ -99,6 +99,14 @@ private[spark] object Config extends Logging { .toSequence .createWithDefault(Nil) + val CONFIG_MAP_MAXSIZE = + ConfigBuilder("spark.kubernetes.configMap.maxSize") + .doc("Max size limit for a config map. This is configurable as per" + + " https://etcd.io/docs/v3.4.0/dev-guide/limit/ on k8s server end.") + .version("3.1.0") + .longConf + .createWithDefault(1572864) // 1.5 MiB + val KUBERNETES_AUTH_DRIVER_CONF_PREFIX = "spark.kubernetes.authenticate.driver" val KUBERNETES_AUTH_EXECUTOR_CONF_PREFIX = "spark.kubernetes.authenticate.executor" val KUBERNETES_AUTH_DRIVER_MOUNTED_CONF_PREFIX = "spark.kubernetes.authenticate.driver.mounted" diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientUtils.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientUtils.scala index 32f630f77d666..4207077677c25 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientUtils.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientUtils.scala @@ -18,15 +18,17 @@ package org.apache.spark.deploy.k8s.submit import java.io.{File, StringWriter} +import java.nio.charset.MalformedInputException import java.util.Properties import scala.collection.JavaConverters._ +import scala.collection.mutable import scala.io.{Codec, Source} import io.fabric8.kubernetes.api.model.{ConfigMap, ConfigMapBuilder, KeyToPath} import org.apache.spark.SparkConf -import org.apache.spark.deploy.k8s.{Constants, KubernetesUtils} +import org.apache.spark.deploy.k8s.{Config, Constants, KubernetesUtils} import org.apache.spark.deploy.k8s.Constants.ENV_SPARK_CONF_DIR import org.apache.spark.internal.Logging @@ -54,8 +56,10 @@ private[spark] object KubernetesClientUtils extends Logging { /** * Build, file -> 'file's content' map of all the selected files in SPARK_CONF_DIR. */ - def buildSparkConfDirFilesMap(configMapName: String, - sparkConf: SparkConf, resolvedPropertiesMap: Map[String, String]): Map[String, String] = { + def buildSparkConfDirFilesMap( + configMapName: String, + sparkConf: SparkConf, + resolvedPropertiesMap: Map[String, String]): Map[String, String] = synchronized { val loadedConfFilesMap = KubernetesClientUtils.loadSparkConfDirFiles(sparkConf) // Add resolved spark conf to the loaded configuration files map. if (resolvedPropertiesMap.nonEmpty) { @@ -90,29 +94,71 @@ private[spark] object KubernetesClientUtils extends Logging { .build() } - private def loadSparkConfDirFiles(conf: SparkConf): Map[String, String] = { + private def orderFilesBySize(confFiles: Seq[File]): Seq[File] = { + val fileToFileSizePairs = confFiles.map(f => (f, f.getName.length + f.length())) + // sort first by name and then by length, so that during tests we have consistent results. + fileToFileSizePairs.sortBy(f => f._1).sortBy(f => f._2).map(_._1) + } + + // exposed for testing + private[submit] def loadSparkConfDirFiles(conf: SparkConf): Map[String, String] = { val confDir = Option(conf.getenv(ENV_SPARK_CONF_DIR)).orElse( conf.getOption("spark.home").map(dir => s"$dir/conf")) + val maxSize = conf.get(Config.CONFIG_MAP_MAXSIZE) if (confDir.isDefined) { - val confFiles = listConfFiles(confDir.get) - logInfo(s"Spark configuration files loaded from $confDir : ${confFiles.mkString(",")}") - confFiles.map { file => - val source = Source.fromFile(file)(Codec.UTF8) - val mapping = (file.getName -> source.mkString) - source.close() - mapping - }.toMap + val confFiles: Seq[File] = listConfFiles(confDir.get, maxSize) + val orderedConfFiles = orderFilesBySize(confFiles) + var truncatedMapSize: Long = 0 + val truncatedMap = mutable.HashMap[String, String]() + val skippedFiles = mutable.HashSet[String]() + var source: Source = Source.fromString("") // init with empty source. + for (file <- orderedConfFiles) { + try { + source = Source.fromFile(file)(Codec.UTF8) + val (fileName, fileContent) = file.getName -> source.mkString + if ((truncatedMapSize + fileName.length + fileContent.length) < maxSize) { + truncatedMap.put(fileName, fileContent) + truncatedMapSize = truncatedMapSize + (fileName.length + fileContent.length) + } else { + skippedFiles.add(fileName) + } + } catch { + case e: MalformedInputException => + logWarning( + s"Unable to read a non UTF-8 encoded file ${file.getAbsolutePath}. Skipping...", e) + None + } finally { + source.close() + } + } + if (truncatedMap.nonEmpty) { + logInfo(s"Spark configuration files loaded from $confDir :" + + s" ${truncatedMap.keys.mkString(",")}") + } + if (skippedFiles.nonEmpty) { + logWarning(s"Skipped conf file(s) ${skippedFiles.mkString(",")}, due to size constraint." + + s" Please see, config: `${Config.CONFIG_MAP_MAXSIZE.key}` for more details.") + } + truncatedMap.toMap } else { Map.empty[String, String] } } - private def listConfFiles(confDir: String): Seq[File] = { - // We exclude all the template files and user provided spark conf or properties. - // As spark properties are resolved in a different step. + private def listConfFiles(confDir: String, maxSize: Long): Seq[File] = { + // At the moment configmaps do not support storing binary content (i.e. skip jar,tar,gzip,zip), + // and configMaps do not allow for size greater than 1.5 MiB(configurable). + // https://etcd.io/docs/v3.4.0/dev-guide/limit/ + def testIfTooLargeOrBinary(f: File): Boolean = (f.length() + f.getName.length > maxSize) || + f.getName.matches(".*\\.(gz|zip|jar|tar)") + + // We exclude all the template files and user provided spark conf or properties, + // Spark properties are resolved in a different step. + def testIfSparkConfOrTemplates(f: File) = f.getName.matches(".*\\.template") || + f.getName.matches("spark.*(conf|properties)") + val fileFilter = (f: File) => { - f.isFile && !(f.getName.endsWith("template") || - f.getName.matches("spark.*(conf|properties)")) + f.isFile && !testIfTooLargeOrBinary(f) && !testIfSparkConfOrTemplates(f) } val confFiles: Seq[File] = { val dir = new File(confDir) diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/ClientSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/ClientSuite.scala index 1a14d524003c0..18d0c00edf2c0 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/ClientSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/ClientSuite.scala @@ -191,25 +191,32 @@ class ClientSuite extends SparkFunSuite with BeforeAndAfter { assert(configMap.getData.get(SPARK_CONF_FILE_NAME).contains("conf2key=conf2value")) } - test("All files from SPARK_CONF_DIR, except templates and spark config " + + test("All files from SPARK_CONF_DIR, " + + "except templates, spark config, binary files and are within size limit, " + "should be populated to pod's configMap.") { def testSetup: (SparkConf, Seq[String]) = { val tempDir = Utils.createTempDir() - val sparkConf = new SparkConf(loadDefaults = false).setSparkHome(tempDir.getAbsolutePath) + val sparkConf = new SparkConf(loadDefaults = false) + .setSparkHome(tempDir.getAbsolutePath) val tempConfDir = new File(s"${tempDir.getAbsolutePath}/conf") tempConfDir.mkdir() // File names - which should not get mounted on the resultant config map. val filteredConfFileNames = - Set("spark-env.sh.template", "spark.properties", "spark-defaults.conf") - val confFileNames = for (i <- 1 to 5) yield s"testConf.$i" ++ + Set("spark-env.sh.template", "spark.properties", "spark-defaults.conf", + "test.gz", "test2.jar", "non_utf8.txt") + val confFileNames = (for (i <- 1 to 5) yield s"testConf.$i") ++ List("spark-env.sh") ++ filteredConfFileNames - val testConfFiles = for (i <- confFileNames) yield { + val testConfFiles = (for (i <- confFileNames) yield { val file = new File(s"${tempConfDir.getAbsolutePath}/$i") - Files.write(file.toPath, "conf1key=conf1value".getBytes(StandardCharsets.UTF_8)) + if (i.startsWith("non_utf8")) { // filling some non-utf-8 binary + Files.write(file.toPath, Array[Byte](0x00.toByte, 0xA1.toByte)) + } else { + Files.write(file.toPath, "conf1key=conf1value".getBytes(StandardCharsets.UTF_8)) + } file.getName - } + }) assert(tempConfDir.listFiles().length == confFileNames.length) val expectedConfFiles: Seq[String] = testConfFiles.filterNot(filteredConfFileNames.contains) (sparkConf, expectedConfFiles) diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientUtilsSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientUtilsSuite.scala new file mode 100644 index 0000000000000..ee672cc041330 --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientUtilsSuite.scala @@ -0,0 +1,79 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.deploy.k8s.submit + +import java.io.File +import java.nio.charset.StandardCharsets +import java.nio.file.Files + +import org.scalatest.BeforeAndAfter + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.deploy.k8s.Config +import org.apache.spark.util.Utils + +class KubernetesClientUtilsSuite extends SparkFunSuite with BeforeAndAfter { + + def testSetup(inputFiles: Map[String, Array[Byte]]): SparkConf = { + val tempDir = Utils.createTempDir() + val sparkConf = new SparkConf(loadDefaults = false) + .setSparkHome(tempDir.getAbsolutePath) + + val tempConfDir = new File(s"${tempDir.getAbsolutePath}/conf") + tempConfDir.mkdir() + for (i <- inputFiles) yield { + val file = new File(s"${tempConfDir.getAbsolutePath}/${i._1}") + Files.write(file.toPath, i._2) + file.getName + } + sparkConf + } + + test("verify load files, loads only allowed files and not the disallowed files.") { + val input: Map[String, Array[Byte]] = Map("test.txt" -> "test123", "z12.zip" -> "zZ", + "rere.jar" -> "@31", "spark.jar" -> "@31", "_test" -> "", "sample.conf" -> "conf") + .map(f => f._1 -> f._2.getBytes(StandardCharsets.UTF_8)) ++ + Map("binary-file.conf" -> Array[Byte](0x00.toByte, 0xA1.toByte)) + val sparkConf = testSetup(input) + val output = KubernetesClientUtils.loadSparkConfDirFiles(sparkConf) + val expectedOutput = Map("test.txt" -> "test123", "sample.conf" -> "conf", "_test" -> "") + assert(output === expectedOutput) + } + + test("verify load files, truncates the content to maxSize, when keys are very large in number.") { + val input = (for (i <- 10000 to 1 by -1) yield (s"testConf.${i}" -> "test123456")).toMap + val sparkConf = testSetup(input.map(f => f._1 -> f._2.getBytes(StandardCharsets.UTF_8))) + .set(Config.CONFIG_MAP_MAXSIZE.key, "60") + val output = KubernetesClientUtils.loadSparkConfDirFiles(sparkConf) + val expectedOutput = Map("testConf.1" -> "test123456", "testConf.2" -> "test123456") + assert(output === expectedOutput) + val output1 = KubernetesClientUtils.loadSparkConfDirFiles( + sparkConf.set(Config.CONFIG_MAP_MAXSIZE.key, "250000")) + assert(output1 === input) + } + + test("verify load files, truncates the content to maxSize, when keys are equal in length.") { + val input = (for (i <- 9 to 1 by -1) yield (s"testConf.${i}" -> "test123456")).toMap + val sparkConf = testSetup(input.map(f => f._1 -> f._2.getBytes(StandardCharsets.UTF_8))) + .set(Config.CONFIG_MAP_MAXSIZE.key, "80") + val output = KubernetesClientUtils.loadSparkConfDirFiles(sparkConf) + val expectedOutput = Map("testConf.1" -> "test123456", "testConf.2" -> "test123456", + "testConf.3" -> "test123456") + assert(output === expectedOutput) + } +} From ff284fb6ac624b2f38ef12f9b840be3077cd27a6 Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Wed, 6 Jan 2021 18:46:20 +0900 Subject: [PATCH 22/29] [SPARK-30681][PYTHON][FOLLOW-UP] Keep the name similar with Scala side in higher order functions ### What changes were proposed in this pull request? This PR is a followup of https://github.com/apache/spark/pull/27406. It fixes the naming to match with Scala side. Note that there are a bit of inconsistency already e.g.) `col`, `e`, `expr` and `column`. This part I did not change but other names like `zero` vs `initialValue` or `col1`/`col2` vs `left`/`right` looks unnecessary. ### Why are the changes needed? To make the usage similar with Scala side, and for consistency. ### Does this PR introduce _any_ user-facing change? No, this is not released yet. ### How was this patch tested? GitHub Actions and Jenkins build will test it out. Closes #31062 from HyukjinKwon/SPARK-30681. Authored-by: HyukjinKwon Signed-off-by: HyukjinKwon --- python/pyspark/sql/functions.py | 16 ++++++++-------- python/pyspark/sql/functions.pyi | 6 +++--- 2 files changed, 11 insertions(+), 11 deletions(-) diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index f612d2d0366f2..c9d24dc668b8e 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -4355,7 +4355,7 @@ def filter(col, f): return _invoke_higher_order_function("ArrayFilter", [col], [f]) -def aggregate(col, zero, merge, finish=None): +def aggregate(col, initialValue, merge, finish=None): """ Applies a binary operator to an initial state and all elements in the array, and reduces this to a single state. The final state is converted into the final result @@ -4372,7 +4372,7 @@ def aggregate(col, zero, merge, finish=None): ---------- col : :class:`Column` or str name of column or expression - zero : :class:`Column` or str + initialValue : :class:`Column` or str initial value. Name of column or expression merge : function a binary function ``(acc: Column, x: Column) -> Column...`` returning expression @@ -4416,19 +4416,19 @@ def aggregate(col, zero, merge, finish=None): if finish is not None: return _invoke_higher_order_function( "ArrayAggregate", - [col, zero], + [col, initialValue], [merge, finish] ) else: return _invoke_higher_order_function( "ArrayAggregate", - [col, zero], + [col, initialValue], [merge] ) -def zip_with(col1, col2, f): +def zip_with(left, right, f): """ Merge two given arrays, element-wise, into a single array using a function. If one array is shorter, nulls are appended at the end to match the length of the longer @@ -4438,9 +4438,9 @@ def zip_with(col1, col2, f): Parameters ---------- - col1 : :class:`Column` or str + left : :class:`Column` or str name of the first column or expression - col2 : :class:`Column` or str + right : :class:`Column` or str name of the second column or expression f : function a binary function ``(x1: Column, x2: Column) -> Column...`` @@ -4471,7 +4471,7 @@ def zip_with(col1, col2, f): |[foo_1, bar_2, 3]| +-----------------+ """ - return _invoke_higher_order_function("ZipWith", [col1, col2], [f]) + return _invoke_higher_order_function("ZipWith", [left, right], [f]) def transform_keys(col, f): diff --git a/python/pyspark/sql/functions.pyi b/python/pyspark/sql/functions.pyi index acb17a2657d00..0cf60c0c26500 100644 --- a/python/pyspark/sql/functions.pyi +++ b/python/pyspark/sql/functions.pyi @@ -237,13 +237,13 @@ def filter(col: ColumnOrName, f: Callable[[Column], Column]) -> Column: ... def filter(col: ColumnOrName, f: Callable[[Column, Column], Column]) -> Column: ... def aggregate( col: ColumnOrName, - zero: ColumnOrName, + initialValue: ColumnOrName, merge: Callable[[Column, Column], Column], finish: Optional[Callable[[Column], Column]] = ..., ) -> Column: ... def zip_with( - col1: ColumnOrName, - ColumnOrName: ColumnOrName, + left: ColumnOrName, + right: ColumnOrName, f: Callable[[Column, Column], Column], ) -> Column: ... def transform_keys( From 0d86a02ffbaf53c403a4c68bac0041e84acb0cdd Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Wed, 6 Jan 2021 20:31:27 +0900 Subject: [PATCH 23/29] [SPARK-34022][DOCS] Support latest mkdocs in SQL built-in function docs ### What changes were proposed in this pull request? This PR adds the support of the latest mkdocs, and makes the sidebar properly show. It works in lower versions too. Before: ![Screen Shot 2021-01-06 at 5 11 56 PM](https://user-images.githubusercontent.com/6477701/103745131-4e7fe400-5042-11eb-9c09-84f9f95e9fb9.png) After: ![Screen Shot 2021-01-06 at 5 10 53 PM](https://user-images.githubusercontent.com/6477701/103745139-5049a780-5042-11eb-8ded-30b6f7ef48aa.png) ### Why are the changes needed? This is a regression in the documentation. ### Does this PR introduce _any_ user-facing change? Technically no. It's not related yet. It fixes the list on the sidebar appears properly. ### How was this patch tested? Manually built the docs via `./sql/create-docs.sh` and `open ./sql/site/index.html` Closes #31061 from HyukjinKwon/SPARK-34022. Authored-by: HyukjinKwon Signed-off-by: HyukjinKwon --- sql/gen-sql-api-docs.py | 1 + 1 file changed, 1 insertion(+) diff --git a/sql/gen-sql-api-docs.py b/sql/gen-sql-api-docs.py index 61328997c1c58..72518504847db 100644 --- a/sql/gen-sql-api-docs.py +++ b/sql/gen-sql-api-docs.py @@ -195,6 +195,7 @@ def generate_sql_api_markdown(jvm, path): """ with open(path, 'w') as mdfile: + mdfile.write("# Built-in Finctions\n\n") for info in _list_function_infos(jvm): name = info.name usage = _make_pretty_usage(info.usage) From 6788304240c416d173ebdb3d544f3361c6b9fe8e Mon Sep 17 00:00:00 2001 From: gengjiaan Date: Wed, 6 Jan 2021 21:14:45 +0900 Subject: [PATCH 24/29] [SPARK-33977][SQL][DOCS] Add doc for "'like any' and 'like all' operators" ### What changes were proposed in this pull request? Add doc for 'like any' and 'like all' operators in sql-ref-syntx-qry-select-like.cmd ### Why are the changes needed? make the usage of 'like any' and 'like all' known to more users ### Does this PR introduce _any_ user-facing change? Yes. Screen Shot 2021-01-06 at 21 10 38 Screen Shot 2021-01-06 at 21 11 06 Screen Shot 2021-01-06 at 21 11 20 ### How was this patch tested? No tests Closes #31008 from beliefer/SPARK-33977. Lead-authored-by: gengjiaan Co-authored-by: beliefer Signed-off-by: Takeshi Yamamuro --- docs/sql-ref-syntax-qry-select-like.md | 60 +++++++++++++++++++++++++- 1 file changed, 59 insertions(+), 1 deletion(-) diff --git a/docs/sql-ref-syntax-qry-select-like.md b/docs/sql-ref-syntax-qry-select-like.md index 6211faa8d529e..3604a9ba1ea02 100644 --- a/docs/sql-ref-syntax-qry-select-like.md +++ b/docs/sql-ref-syntax-qry-select-like.md @@ -21,12 +21,14 @@ license: | ### Description -A LIKE predicate is used to search for a specific pattern. +A LIKE predicate is used to search for a specific pattern. This predicate also supports multiple patterns with quantifiers include `ANY`, `SOME` and `ALL`. ### Syntax ```sql [ NOT ] { LIKE search_pattern [ ESCAPE esc_char ] | [ RLIKE | REGEXP ] regex_pattern } + +[ NOT ] { LIKE quantifiers ( search_pattern [ , ... ]) } ``` ### Parameters @@ -45,6 +47,10 @@ A LIKE predicate is used to search for a specific pattern. * **regex_pattern** Specifies a regular expression search pattern to be searched by the `RLIKE` or `REGEXP` clause. + +* **quantifiers** + + Specifies the predicate quantifiers include `ANY`, `SOME` and `ALL`. `ANY` or `SOME` means if one of the patterns matches the input, then return true; `ALL` means if all the patterns matches the input, then return true. ### Examples @@ -111,6 +117,58 @@ SELECT * FROM person WHERE name LIKE '%$_%' ESCAPE '$'; +---+------+---+ |500|Evan_W| 16| +---+------+---+ + +SELECT * FROM person WHERE name LIKE ALL ('%an%', '%an'); ++---+----+----+ +| id|name| age| ++---+----+----+ +|400| Dan| 50| ++---+----+----+ + +SELECT * FROM person WHERE name LIKE ANY ('%an%', '%an'); ++---+------+---+ +| id| name|age| ++---+------+---+ +|400| Dan| 50| +|500|Evan_W| 16| ++---+------+---+ + +SELECT * FROM person WHERE name LIKE SOME ('%an%', '%an'); ++---+------+---+ +| id| name|age| ++---+------+---+ +|400| Dan| 50| +|500|Evan_W| 16| ++---+------+---+ + +SELECT * FROM person WHERE name NOT LIKE ALL ('%an%', '%an'); ++---+----+----+ +| id|name| age| ++---+----+----+ +|100|John| 30| +|200|Mary|null| +|300|Mike| 80| ++---+----+----+ + +SELECT * FROM person WHERE name NOT LIKE ANY ('%an%', '%an'); ++---+------+----+ +| id| name| age| ++---+------+----+ +|100| John| 30| +|200| Mary|null| +|300| Mike| 80| +|500|Evan_W| 16| ++---+------+----+ + +SELECT * FROM person WHERE name NOT LIKE SOME ('%an%', '%an'); ++---+------+----+ +| id| name| age| ++---+------+----+ +|100| John| 30| +|200| Mary|null| +|300| Mike| 80| +|500|Evan_W| 16| ++---+------+----+ ``` ### Related Statements From 3cdc4ef5b41ce1254610436a8721ea517124d62e Mon Sep 17 00:00:00 2001 From: angerszhu Date: Wed, 6 Jan 2021 13:45:48 +0000 Subject: [PATCH 25/29] [SPARK-32685][SQL][FOLLOW-UP] Update migration guide about change default filed.delim to '\t' when user specifies serde ### What changes were proposed in this pull request? Update migration guide according to https://github.com/apache/spark/pull/30942#issuecomment-755054562 ### Why are the changes needed? update migration guide. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Not need Closes #31051 from AngersZhuuuu/SPARK-32685-FOLLOW-UP. Authored-by: angerszhu Signed-off-by: Wenchen Fan --- docs/sql-migration-guide.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/sql-migration-guide.md b/docs/sql-migration-guide.md index bd54554baa09d..8cf1a9c6f7017 100644 --- a/docs/sql-migration-guide.md +++ b/docs/sql-migration-guide.md @@ -30,7 +30,7 @@ license: | - In Spark 3.2, `ALTER TABLE .. RENAME TO PARTITION` throws `PartitionAlreadyExistsException` instead of `AnalysisException` for tables from Hive external when the target partition already exists. - - In Spark 3.2, script transform default FIELD DELIMIT is `\u0001` for no serde mode. In Spark 3.1 or earlier, the default FIELD DELIMIT is `\t`. + - In Spark 3.2, script transform default FIELD DELIMIT is `\u0001` for no serde mode, serde property `field.delim` is `\t` for Hive serde mode when user specifies serde. In Spark 3.1 or earlier, the default FIELD DELIMIT is `\t`, serde property `field.delim` is `\u0001` for Hive serde mode when user specifies serde. ## Upgrading from Spark SQL 3.0 to 3.1 From a0269bb419a37c31850e02884385b889cd153133 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Wed, 6 Jan 2021 09:28:22 -0800 Subject: [PATCH 26/29] [SPARK-34022][DOCS][FOLLOW-UP] Fix typo in SQL built-in function docs ### What changes were proposed in this pull request? This PR is a follow-up of #31061. It fixes a typo in a document: `Finctions` -> `Functions` ### Why are the changes needed? Make the change better documented. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? N/A Closes #31069 from kiszk/SPARK-34022-followup. Authored-by: Kazuaki Ishizaki Signed-off-by: Dongjoon Hyun --- sql/gen-sql-api-docs.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/gen-sql-api-docs.py b/sql/gen-sql-api-docs.py index 72518504847db..2f734093b106c 100644 --- a/sql/gen-sql-api-docs.py +++ b/sql/gen-sql-api-docs.py @@ -195,7 +195,7 @@ def generate_sql_api_markdown(jvm, path): """ with open(path, 'w') as mdfile: - mdfile.write("# Built-in Finctions\n\n") + mdfile.write("# Built-in Functions\n\n") for info in _list_function_infos(jvm): name = info.name usage = _make_pretty_usage(info.usage) From 8bb70bf0d646f6d54d17690d23ee935e452e747e Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Wed, 6 Jan 2021 12:59:47 -0800 Subject: [PATCH 27/29] [SPARK-34029][SQL][TESTS] Add OrcEncryptionSuite and FakeKeyProvider ### What changes were proposed in this pull request? This PR aims to add a basis for columnar encryption test framework by add `OrcEncryptionSuite` and `FakeKeyProvider`. Please note that we will improve more in both Apache Spark and Apache ORC in Apache Spark 3.2.0 timeframe. ### Why are the changes needed? Apache ORC 1.6 supports columnar encryption. ### Does this PR introduce _any_ user-facing change? No. This is for a test case. ### How was this patch tested? Pass the newly added test suite. Closes #31065 from dongjoon-hyun/SPARK-34029. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- project/SparkBuild.scala | 1 + .../datasources/orc/FakeKeyProvider.java | 144 ++++++++++++++++++ ...pache.hadoop.crypto.key.KeyProviderFactory | 16 ++ .../datasources/orc/OrcEncryptionSuite.scala | 98 ++++++++++++ 4 files changed, 259 insertions(+) create mode 100644 sql/core/src/test/java/test/org/apache/spark/sql/execution/datasources/orc/FakeKeyProvider.java create mode 100644 sql/core/src/test/resources/META-INF/services/org.apache.hadoop.crypto.key.KeyProviderFactory create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcEncryptionSuite.scala diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 668701be0ae98..f126ee35efcca 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -489,6 +489,7 @@ object SparkParallelTestGrouping { "org.apache.spark.sql.catalyst.expressions.HashExpressionsSuite", "org.apache.spark.sql.catalyst.expressions.CastSuite", "org.apache.spark.sql.catalyst.expressions.MathExpressionsSuite", + "org.apache.spark.sql.execution.datasources.orc.OrcEncryptionSuite", "org.apache.spark.sql.hive.HiveExternalCatalogSuite", "org.apache.spark.sql.hive.StatisticsSuite", "org.apache.spark.sql.hive.client.VersionsSuite", diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/execution/datasources/orc/FakeKeyProvider.java b/sql/core/src/test/java/test/org/apache/spark/sql/execution/datasources/orc/FakeKeyProvider.java new file mode 100644 index 0000000000000..c48543802eb33 --- /dev/null +++ b/sql/core/src/test/java/test/org/apache/spark/sql/execution/datasources/orc/FakeKeyProvider.java @@ -0,0 +1,144 @@ +/* + * 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 test.org.apache.spark.sql.execution.datasources.orc; + +import java.io.IOException; +import java.net.URI; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.crypto.key.KeyProvider; +import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension; +import org.apache.hadoop.crypto.key.KeyProviderFactory; +import org.apache.hadoop.crypto.key.kms.KMSClientProvider; + +/** + * A Hadoop KeyProvider that lets us test the interaction + * with the Hadoop code. + * + * https://github.com/apache/orc/blob/rel/release-1.6.6/java/tools/src/test/org/apache/orc/impl/FakeKeyProvider.java + * + * This file intentionally keeps the original file except + * (1) package name, (2) import order, (3) a few indentation + */ +public class FakeKeyProvider extends KeyProvider { + // map from key name to metadata + private final Map keyMetdata = new HashMap<>(); + // map from key version name to material + private final Map keyVersions = new HashMap<>(); + + public FakeKeyProvider(Configuration conf) { + super(conf); + } + + @Override + public KeyVersion getKeyVersion(String name) { + return keyVersions.get(name); + } + + @Override + public List getKeys() { + return new ArrayList<>(keyMetdata.keySet()); + } + + @Override + public List getKeyVersions(String name) { + List result = new ArrayList<>(); + Metadata meta = getMetadata(name); + for(int v=0; v < meta.getVersions(); ++v) { + String versionName = buildVersionName(name, v); + KeyVersion material = keyVersions.get(versionName); + if (material != null) { + result.add(material); + } + } + return result; + } + + @Override + public Metadata getMetadata(String name) { + return keyMetdata.get(name); + } + + @Override + public KeyVersion createKey(String name, byte[] bytes, Options options) { + String versionName = buildVersionName(name, 0); + keyMetdata.put(name, new TestMetadata(options.getCipher(), + options.getBitLength(), 1)); + KeyVersion result = new KMSClientProvider.KMSKeyVersion(name, versionName, bytes); + keyVersions.put(versionName, result); + return result; + } + + @Override + public void deleteKey(String name) { + throw new UnsupportedOperationException("Can't delete keys"); + } + + @Override + public KeyVersion rollNewVersion(String name, byte[] bytes) { + TestMetadata key = keyMetdata.get(name); + String versionName = buildVersionName(name, key.addVersion()); + KeyVersion result = new KMSClientProvider.KMSKeyVersion(name, versionName, + bytes); + keyVersions.put(versionName, result); + return result; + } + + @Override + public void flush() { + // Nothing + } + + static class TestMetadata extends KeyProvider.Metadata { + + TestMetadata(String cipher, int bitLength, int versions) { + super(cipher, bitLength, null, null, null, versions); + } + + public int addVersion() { + return super.addVersion(); + } + } + + public static class Factory extends KeyProviderFactory { + + @Override + public KeyProvider createProvider(URI uri, Configuration conf) throws IOException { + if ("test".equals(uri.getScheme())) { + KeyProvider provider = new FakeKeyProvider(conf); + // populate a couple keys into the provider + byte[] piiKey = new byte[]{0,1,2,3,4,5,6,7,8,9,0xa,0xb,0xc,0xd,0xe,0xf}; + org.apache.hadoop.crypto.key.KeyProvider.Options aes128 = new KeyProvider.Options(conf); + provider.createKey("pii", piiKey, aes128); + byte[] piiKey2 = new byte[]{0x10,0x11,0x12,0x13,0x14,0x15,0x16,0x17, + 0x18,0x19,0x1a,0x1b,0x1c,0x1d,0x1e,0x1f}; + provider.rollNewVersion("pii", piiKey2); + byte[] secretKey = new byte[]{0x20,0x21,0x22,0x23,0x24,0x25,0x26,0x27, + 0x28,0x29,0x2a,0x2b,0x2c,0x2d,0x2e,0x2f}; + provider.createKey("secret", secretKey, aes128); + return KeyProviderCryptoExtension.createKeyProviderCryptoExtension(provider); + } + return null; + } + } +} diff --git a/sql/core/src/test/resources/META-INF/services/org.apache.hadoop.crypto.key.KeyProviderFactory b/sql/core/src/test/resources/META-INF/services/org.apache.hadoop.crypto.key.KeyProviderFactory new file mode 100644 index 0000000000000..f436622b5fb42 --- /dev/null +++ b/sql/core/src/test/resources/META-INF/services/org.apache.hadoop.crypto.key.KeyProviderFactory @@ -0,0 +1,16 @@ +# 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. + +test.org.apache.spark.sql.execution.datasources.orc.FakeKeyProvider$Factory diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcEncryptionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcEncryptionSuite.scala new file mode 100644 index 0000000000000..fac3cef5801dd --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcEncryptionSuite.scala @@ -0,0 +1,98 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.datasources.orc + +import org.apache.spark.sql.Row +import org.apache.spark.sql.test.SharedSparkSession + +class OrcEncryptionSuite extends OrcTest with SharedSparkSession { + import testImplicits._ + + val originalData = Seq(("123456789", "dongjoon@apache.org", "Dongjoon Hyun")) + val rowDataWithoutKey = + Row(null, "841626795E7D351555B835A002E3BF10669DE9B81C95A3D59E10865AC37EA7C3", "Dongjoon Hyun") + + test("Write and read an encrypted file") { + val df = originalData.toDF("ssn", "email", "name") + + withTempPath { dir => + val path = dir.getAbsolutePath + withSQLConf( + "hadoop.security.key.provider.path" -> "test:///", + "orc.key.provider" -> "hadoop", + "orc.encrypt" -> "pii:ssn,email", + "orc.mask" -> "nullify:ssn;sha256:email") { + df.write.mode("overwrite").orc(path) + checkAnswer(spark.read.orc(path), df) + } + + withSQLConf( + "orc.key.provider" -> "memory", + "orc.encrypt" -> "pii:ssn,email", + "orc.mask" -> "nullify:ssn;sha256:email") { + checkAnswer(spark.read.orc(path), rowDataWithoutKey) + } + } + } + + test("Write and read an encrypted table") { + val df = originalData.toDF("ssn", "email", "name") + + withTempPath { dir => + val path = dir.getAbsolutePath + withTable("encrypted") { + sql( + s""" + |CREATE TABLE encrypted ( + | ssn STRING, + | email STRING, + | name STRING + |) + |USING ORC + |LOCATION "$path" + |OPTIONS ( + | hadoop.security.key.provider.path "test:///", + | orc.key.provider "hadoop", + | orc.encrypt "pii:ssn,email", + | orc.mask "nullify:ssn;sha256:email" + |) + |""".stripMargin) + sql("INSERT INTO encrypted VALUES('123456789', 'dongjoon@apache.org', 'Dongjoon Hyun')") + checkAnswer(sql("SELECT * FROM encrypted"), df) + } + withTable("normal") { + sql( + s""" + |CREATE TABLE normal ( + | ssn STRING, + | email STRING, + | name STRING + |) + |USING ORC + |LOCATION "$path" + |OPTIONS ( + | orc.key.provider "memory", + | orc.encrypt "pii:ssn,email", + | orc.mask "nullify:ssn;sha256:email" + |) + |""".stripMargin) + checkAnswer(sql("SELECT * FROM normal"), rowDataWithoutKey) + } + } + } +} From f9daf035f473fea12a2ee67428db8d78f29973d5 Mon Sep 17 00:00:00 2001 From: ulysses-you Date: Wed, 6 Jan 2021 17:22:14 -0800 Subject: [PATCH 28/29] [SPARK-33806][SQL][FOLLOWUP] Fold RepartitionExpression num partition should check if partition expression is empty ### What changes were proposed in this pull request? Add check partition expressions is empty. ### Why are the changes needed? We should keep `spark.range(1).hint("REPARTITION_BY_RANGE")` has default shuffle number instead of 1. ### Does this PR introduce _any_ user-facing change? Yes. ### How was this patch tested? Add test. Closes #31074 from ulysses-you/SPARK-33806-FOLLOWUP. Authored-by: ulysses-you Signed-off-by: Dongjoon Hyun --- .../plans/logical/basicLogicalOperators.scala | 2 +- .../scala/org/apache/spark/sql/SQLQuerySuite.scala | 11 +++++++++++ 2 files changed, 12 insertions(+), 1 deletion(-) 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 ee7db7ae83542..9e06f9bec7830 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 @@ -1050,7 +1050,7 @@ case class RepartitionByExpression( val numPartitions = if (optNumPartitions.nonEmpty) { optNumPartitions.get } else { - if (partitionExpressions.forall(_.foldable)) { + if (partitionExpressions.nonEmpty && partitionExpressions.forall(_.foldable)) { 1 } else { SQLConf.get.numShufflePartitions diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 44f3c3449ddda..3f55a88f19505 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -3768,6 +3768,17 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark } } + test("Fold RepartitionExpression num partition should check if partition expression is empty") { + withSQLConf((SQLConf.SHUFFLE_PARTITIONS.key, "5")) { + val df = spark.range(1).hint("REPARTITION_BY_RANGE") + val plan = df.queryExecution.optimizedPlan + val res = plan.collect { + case r: RepartitionByExpression if r.numPartitions == 5 => true + } + assert(res.nonEmpty) + } + } + test("SPARK-33593: Vector reader got incorrect data with binary partition value") { Seq("false", "true").foreach(value => { withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> value) { From 9b5df2afaa5df85f149ccf73b7a6b78ab0f393bc Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Wed, 6 Jan 2021 20:19:16 -0800 Subject: [PATCH 29/29] [SPARK-34036][DOCS] Update ORC data source documentation ### What changes were proposed in this pull request? This PR aims to update SQL documentation about ORC data sources. New structure looks like the following. - ORC Implementation - Vectorized Reader - Schema Merging - Zstandard - Bloom Filters - Columnar Encryption - Hive metastore ORC table conversion - Configuration ### Why are the changes needed? This document is not up-to-date. Apache Spark 3.2.0 can utilize new improvements from Apache ORC 1.6.6. ### Does this PR introduce _any_ user-facing change? No, this is a documentation. ### How was this patch tested? Manual. ``` SKIP_API=1 jekyll build ``` --- **BEFORE** ![Screen Shot 2021-01-06 at 5 08 19 PM](https://user-images.githubusercontent.com/9700541/103838399-d0bbd880-5041-11eb-8757-297728d2793f.png) --- **AFTER** ![Screen Shot 2021-01-06 at 7 03 38 PM](https://user-images.githubusercontent.com/9700541/103845972-0963ae00-5052-11eb-905e-8e8b335c760a.png) ![Screen Shot 2021-01-06 at 7 03 49 PM](https://user-images.githubusercontent.com/9700541/103845971-08cb1780-5052-11eb-9b2a-d3acfa4b9278.png) ![Screen Shot 2021-01-06 at 7 03 59 PM](https://user-images.githubusercontent.com/9700541/103845970-08328100-5052-11eb-8982-7079fd7b0efc.png) ![Screen Shot 2021-01-06 at 7 04 10 PM](https://user-images.githubusercontent.com/9700541/103845968-08328100-5052-11eb-9ef5-db99c7cc64d3.png) ![Screen Shot 2021-01-06 at 7 04 16 PM](https://user-images.githubusercontent.com/9700541/103845963-07015400-5052-11eb-955f-8126d417e8aa.png) Closes #31075 from dongjoon-hyun/SPARK-34036. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- docs/sql-data-sources-orc.md | 135 +++++++++++++++++++++++++++++++++-- 1 file changed, 129 insertions(+), 6 deletions(-) diff --git a/docs/sql-data-sources-orc.md b/docs/sql-data-sources-orc.md index 4c4b3b1eee8c2..f5c9677c343dc 100644 --- a/docs/sql-data-sources-orc.md +++ b/docs/sql-data-sources-orc.md @@ -19,12 +19,115 @@ license: | limitations under the License. --- -Since Spark 2.3, Spark supports a vectorized ORC reader with a new ORC file format for ORC files. -To do that, the following configurations are newly added. The vectorized reader is used for the -native ORC tables (e.g., the ones created using the clause `USING ORC`) when `spark.sql.orc.impl` -is set to `native` and `spark.sql.orc.enableVectorizedReader` is set to `true`. For the Hive ORC -serde tables (e.g., the ones created using the clause `USING HIVE OPTIONS (fileFormat 'ORC')`), -the vectorized reader is used when `spark.sql.hive.convertMetastoreOrc` is also set to `true`. +* Table of contents +{:toc} + +[Apache ORC](https://orc.apache.org) is a columnar format which has more advanced features like native zstd compression, bloom filter and columnar encryption. + +### ORC Implementation + +Spark supports two ORC implementations (`native` and `hive`) which is controlled by `spark.sql.orc.impl`. +Two implementations share most functionalities with different design goals. +- `native` implementation is designed to follow Spark's data source behavior like `Parquet`. +- `hive` implementation is designed to follow Hive's behavior and uses Hive SerDe. + +For example, historically, `native` implementation handles `CHAR/VARCHAR` with Spark's native `String` while `hive` implementation handles it via Hive `CHAR/VARCHAR`. The query results are different. Since Spark 3.1.0, [SPARK-33480](https://issues.apache.org/jira/browse/SPARK-33480) removes this difference by supporting `CHAR/VARCHAR` from Spark-side. + +### Vectorized Reader + +`native` implementation supports a vectorized ORC reader and has been the default ORC implementaion since Spark 2.3. +The vectorized reader is used for the native ORC tables (e.g., the ones created using the clause `USING ORC`) when `spark.sql.orc.impl` is set to `native` and `spark.sql.orc.enableVectorizedReader` is set to `true`. +For the Hive ORC serde tables (e.g., the ones created using the clause `USING HIVE OPTIONS (fileFormat 'ORC')`), +the vectorized reader is used when `spark.sql.hive.convertMetastoreOrc` is also set to `true`, and is turned on by default. + +### Schema Merging + +Like Protocol Buffer, Avro, and Thrift, ORC also supports schema evolution. Users can start with +a simple schema, and gradually add more columns to the schema as needed. In this way, users may end +up with multiple ORC files with different but mutually compatible schemas. The ORC data +source is now able to automatically detect this case and merge schemas of all these files. + +Since schema merging is a relatively expensive operation, and is not a necessity in most cases, we +turned it off by default . You may enable it by + +1. setting data source option `mergeSchema` to `true` when reading ORC files, or +2. setting the global SQL option `spark.sql.orc.mergeSchema` to `true`. + +### Zstandard + +Spark supports both Hadoop 2 and 3. Since Spark 3.2, you can take advantage +of Zstandard compression in ORC files on both Hadoop versions. +Please see [Zstandard](https://facebook.github.io/zstd/) for the benefits. + +
+
+ +{% highlight sql %} +CREATE TABLE compressed ( + key STRING, + value STRING +) +USING ORC +OPTIONS ( + compression 'zstd' +) +{% endhighlight %} +
+
+ +### Bloom Filters + +You can control bloom filters and dictionary encodings for ORC data sources. The following ORC example will create bloom filter and use dictionary encoding only for `favorite_color`. To find more detailed information about the extra ORC options, visit the official Apache ORC websites. + +
+
+ +{% highlight sql %} +CREATE TABLE users_with_options ( + name STRING, + favorite_color STRING, + favorite_numbers array +) +USING ORC +OPTIONS ( + orc.bloom.filter.columns 'favorite_color', + orc.dictionary.key.threshold '1.0', + orc.column.encoding.direct 'name' +) +{% endhighlight %} +
+
+ +### Columnar Encryption + +Since Spark 3.2, columnar encryption is supported for ORC tables with Apache ORC 1.6. +The following example is using Hadoop KMS as a key provider with the given location. +Please visit [Apache Hadoop KMS](https://hadoop.apache.org/docs/current/hadoop-kms/index.html) for the detail. + +
+
+{% highlight sql %} +CREATE TABLE encrypted ( + ssn STRING, + email STRING, + name STRING +) +USING ORC +OPTIONS ( + hadoop.security.key.provider.path "kms://http@localhost:9600/kms", + orc.key.provider "hadoop", + orc.encrypt "pii:ssn,email", + orc.mask "nullify:ssn;sha256:email" +) +{% endhighlight %} +
+
+ +### Hive metastore ORC table conversion + +When reading from Hive metastore ORC tables and inserting to Hive metastore ORC tables, Spark SQL will try to use its own ORC support instead of Hive SerDe for better performance. For CTAS statement, only non-partitioned Hive metastore ORC tables are converted. This behavior is controlled by the `spark.sql.hive.convertMetastoreOrc` configuration, and is turned on by default. + +### Configuration @@ -48,4 +151,24 @@ the vectorized reader is used when `spark.sql.hive.convertMetastoreOrc` is also + + + + + + + + + + + +
Property NameDefaultMeaningSince Version
2.3.0
spark.sql.orc.mergeSchemafalse +

+ When true, the ORC data source merges schemas collected from all data files, + otherwise the schema is picked from a random data file. +

+
3.0.0
spark.sql.hive.convertMetastoreOrctrue + When set to false, Spark SQL will use the Hive SerDe for ORC tables instead of the built in + support. + 2.0.0